From 9787b3a1eeae3feefcdd53f5a4b7a5f831e18579 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 11 Feb 2019 14:19:56 +0300 Subject: [PATCH 01/40] Added Field type AggregateFunctionState. --- dbms/src/Columns/ColumnAggregateFunction.cpp | 12 ++++---- dbms/src/Common/FieldVisitors.cpp | 15 ++++++++++ dbms/src/Common/FieldVisitors.h | 29 ++++++++++++++++++++ dbms/src/Core/Field.cpp | 24 ++++++++++++++++ dbms/src/Core/Field.h | 18 +++++++++++- 5 files changed, 91 insertions(+), 7 deletions(-) diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index 7d3e001998a..ae72d263a15 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -260,9 +260,9 @@ MutableColumnPtr ColumnAggregateFunction::cloneEmpty() const Field ColumnAggregateFunction::operator[](size_t n) const { - Field field = String(); + Field field = AggregateFunctionStateData(); { - WriteBufferFromString buffer(field.get()); + WriteBufferFromString buffer(field.get().toUnderType()); func->serialize(data[n], buffer); } return field; @@ -272,7 +272,7 @@ void ColumnAggregateFunction::get(size_t n, Field & res) const { res = String(); { - WriteBufferFromString buffer(res.get()); + WriteBufferFromString buffer(res.get().toUnderType()); func->serialize(data[n], buffer); } } @@ -343,7 +343,7 @@ void ColumnAggregateFunction::insert(const Field & x) ensureOwnership(); Arena & arena = createOrGetArena(); pushBackAndCreateState(data, arena, func.get()); - ReadBufferFromString read_buffer(x.get()); + ReadBufferFromString read_buffer(x.get().toUnderType()); func->deserialize(data.back(), read_buffer, &arena); } @@ -465,12 +465,12 @@ void ColumnAggregateFunction::getExtremes(Field & min, Field & max) const AlignedBuffer place_buffer(func->sizeOfData(), func->alignOfData()); AggregateDataPtr place = place_buffer.data(); - String serialized; + AggregateFunctionStateData serialized; func->create(place); try { - WriteBufferFromString buffer(serialized); + WriteBufferFromString buffer(serialized.toUnderType()); func->serialize(place, buffer); } catch (...) diff --git a/dbms/src/Common/FieldVisitors.cpp b/dbms/src/Common/FieldVisitors.cpp index e2b855c8f83..e10095fed90 100644 --- a/dbms/src/Common/FieldVisitors.cpp +++ b/dbms/src/Common/FieldVisitors.cpp @@ -89,6 +89,12 @@ String FieldVisitorDump::operator() (const Tuple & x_def) const return wb.str(); } +String FieldVisitorDump::operator() (const AggregateFunctionStateData & x) const +{ + WriteBufferFromOwnString wb; + writeQuoted(x, wb); + return wb.str(); +} /** In contrast to writeFloatText (and writeQuoted), * even if number looks like integer after formatting, prints decimal point nevertheless (for example, Float64(1) is printed as 1.). @@ -121,6 +127,7 @@ String FieldVisitorToString::operator() (const DecimalField & x) cons String FieldVisitorToString::operator() (const DecimalField & x) const { return formatQuoted(x); } String FieldVisitorToString::operator() (const DecimalField & x) const { return formatQuoted(x); } String FieldVisitorToString::operator() (const UInt128 & x) const { return formatQuoted(UUID(x)); } +String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const { return formatQuoted(x); } String FieldVisitorToString::operator() (const Array & x) const { @@ -231,5 +238,13 @@ void FieldVisitorHash::operator() (const DecimalField & x) const hash.update(x); } +void FieldVisitorHash::operator() (const AggregateFunctionStateData & x) const +{ + UInt8 type = Field::Types::AggregateFunctionState; + hash.update(type); + hash.update(x.toUnderType().size()); + hash.update(x.toUnderType().data(), x.toUnderType().size()); +} + } diff --git a/dbms/src/Common/FieldVisitors.h b/dbms/src/Common/FieldVisitors.h index 19b4e583338..30aa700f80d 100644 --- a/dbms/src/Common/FieldVisitors.h +++ b/dbms/src/Common/FieldVisitors.h @@ -138,6 +138,7 @@ public: String operator() (const DecimalField & x) const; String operator() (const DecimalField & x) const; String operator() (const DecimalField & x) const; + String operator() (const AggregateFunctionStateData & x) const; }; @@ -156,6 +157,7 @@ public: String operator() (const DecimalField & x) const; String operator() (const DecimalField & x) const; String operator() (const DecimalField & x) const; + String operator() (const AggregateFunctionStateData & x) const; }; @@ -201,6 +203,11 @@ public: else return x.getValue() / x.getScaleMultiplier(); } + + T operator() (const AggregateFunctionStateData &) const + { + throw Exception("Cannot convert String to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE); + } }; @@ -222,6 +229,7 @@ public: void operator() (const DecimalField & x) const; void operator() (const DecimalField & x) const; void operator() (const DecimalField & x) const; + void operator() (const AggregateFunctionStateData & x) const; }; @@ -321,6 +329,16 @@ public: template bool operator() (const Int64 & l, const DecimalField & r) const { return DecimalField(l, 0) == r; } template bool operator() (const Float64 & l, const DecimalField & r) const { return cantCompare(l, r); } + template + bool operator() (const AggregateFunctionStateData & l, const T & r) const + { + if constexpr (std::is_same_v) + return l == r; + if constexpr (std::is_same_v) + return stringToUUID(l.toUnderType()) == r; + return cantCompare(l, r); + } + private: template bool cantCompare(const T &, const U &) const @@ -419,6 +437,16 @@ public: template bool operator() (const Int64 & l, const DecimalField & r) const { return DecimalField(l, 0) < r; } template bool operator() (const Float64 &, const DecimalField &) const { return false; } + template + bool operator() (const AggregateFunctionStateData & l, const T & r) const + { + if constexpr (std::is_same_v) + return l < r; + if constexpr (std::is_same_v) + return stringToUUID(l.toUnderType()) < r; + return cantCompare(l, r); + } + private: template bool cantCompare(const T &, const U &) const @@ -447,6 +475,7 @@ public: bool operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); } bool operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); } bool operator() (UInt128 &) const { throw Exception("Cannot sum UUIDs", ErrorCodes::LOGICAL_ERROR); } + bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot sum AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); } template bool operator() (DecimalField & x) const diff --git a/dbms/src/Core/Field.cpp b/dbms/src/Core/Field.cpp index c195652d051..7411d17efc1 100644 --- a/dbms/src/Core/Field.cpp +++ b/dbms/src/Core/Field.cpp @@ -75,6 +75,13 @@ namespace DB x.push_back(value); break; } + case Field::Types::AggregateFunctionState: + { + AggregateFunctionStateData value; + DB::readStringBinary(value, buf); + x.push_back(value); + break; + } } } } @@ -128,6 +135,11 @@ namespace DB DB::writeBinary(get(*it), buf); break; } + case Field::Types::AggregateFunctionState: + { + DB::writeStringBinary(get(*it), buf); + break; + } } } } @@ -209,6 +221,13 @@ namespace DB x.push_back(value); break; } + case Field::Types::AggregateFunctionState: + { + AggregateFunctionStateData value; + DB::readStringBinary(value, buf); + x.push_back(value); + break; + } } } } @@ -262,6 +281,11 @@ namespace DB DB::writeBinary(get(*it), buf); break; } + case Field::Types::AggregateFunctionState: + { + DB::writeStringBinary(get(*it), buf); + break; + } } } } diff --git a/dbms/src/Core/Field.h b/dbms/src/Core/Field.h index 47242825f86..583b3303c56 100644 --- a/dbms/src/Core/Field.h +++ b/dbms/src/Core/Field.h @@ -30,6 +30,9 @@ using Array = std::vector; using TupleBackend = std::vector; STRONG_TYPEDEF(TupleBackend, Tuple) /// Array and Tuple are different types with equal representation inside Field. +using AggregateFunctionStateDataBackend = String; +STRONG_TYPEDEF(AggregateFunctionStateDataBackend, AggregateFunctionStateData) + template bool decimalEqual(T x, T y, UInt32 x_scale, UInt32 y_scale); template bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale); template bool decimalLessOrEqual(T x, T y, UInt32 x_scale, UInt32 y_scale); @@ -131,6 +134,7 @@ public: Decimal32 = 19, Decimal64 = 20, Decimal128 = 21, + AggregateFunctionState = 22, }; static const int MIN_NON_POD = 16; @@ -151,6 +155,7 @@ public: case Decimal32: return "Decimal32"; case Decimal64: return "Decimal64"; case Decimal128: return "Decimal128"; + case AggregateFunctionState: return "AggregateFunctionState"; } throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); @@ -325,6 +330,7 @@ public: case Types::Decimal32: return get>() < rhs.get>(); case Types::Decimal64: return get>() < rhs.get>(); case Types::Decimal128: return get>() < rhs.get>(); + case Types::AggregateFunctionState: return get() < rhs.get(); } throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); @@ -356,6 +362,7 @@ public: case Types::Decimal32: return get>() <= rhs.get>(); case Types::Decimal64: return get>() <= rhs.get>(); case Types::Decimal128: return get>() <= rhs.get>(); + case Types::AggregateFunctionState: return get() <= rhs.get(); } throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); @@ -385,6 +392,7 @@ public: case Types::Decimal32: return get>() == rhs.get>(); case Types::Decimal64: return get>() == rhs.get>(); case Types::Decimal128: return get>() == rhs.get>(); + case Types::AggregateFunctionState: return get() == rhs.get(); } throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); @@ -398,7 +406,7 @@ public: private: std::aligned_union_t, DecimalField, DecimalField + DecimalField, DecimalField, DecimalField, AggregateFunctionStateData > storage; Types::Which which; @@ -449,6 +457,7 @@ private: case Types::Decimal32: f(field.template get>()); return; case Types::Decimal64: f(field.template get>()); return; case Types::Decimal128: f(field.template get>()); return; + case Types::AggregateFunctionState: f(field.template get()); return; } } @@ -501,6 +510,9 @@ private: case Types::Tuple: destroy(); break; + case Types::AggregateFunctionState: + destroy(); + break; default: break; } @@ -531,6 +543,7 @@ template <> struct Field::TypeToEnum { static const Types::Which value template <> struct Field::TypeToEnum>{ static const Types::Which value = Types::Decimal32; }; template <> struct Field::TypeToEnum>{ static const Types::Which value = Types::Decimal64; }; template <> struct Field::TypeToEnum>{ static const Types::Which value = Types::Decimal128; }; +template <> struct Field::TypeToEnum{ static const Types::Which value = Types::AggregateFunctionState; }; template <> struct Field::EnumToType { using Type = Null; }; template <> struct Field::EnumToType { using Type = UInt64; }; @@ -544,6 +557,7 @@ template <> struct Field::EnumToType { using Type = Tuple template <> struct Field::EnumToType { using Type = DecimalField; }; template <> struct Field::EnumToType { using Type = DecimalField; }; template <> struct Field::EnumToType { using Type = DecimalField; }; +template <> struct Field::EnumToType { using Type = DecimalField; }; template @@ -616,6 +630,8 @@ template <> struct NearestFieldTypeImpl { using Type = Tuple; }; template <> struct NearestFieldTypeImpl { using Type = UInt64; }; template <> struct NearestFieldTypeImpl { using Type = Null; }; +template <> struct NearestFieldTypeImpl { using Type = AggregateFunctionStateData; }; + template using NearestFieldType = typename NearestFieldTypeImpl::Type; From 1de1192aa8514feb4809813976a4183cafc5b5c0 Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 11 Feb 2019 17:19:09 +0300 Subject: [PATCH 02/40] move asterisks extraction to TranslateQualifiedNamesVisitor (before column names normalisation) --- dbms/src/Interpreters/ActionsVisitor.cpp | 1 + .../PredicateExpressionsOptimizer.cpp | 6 +- dbms/src/Interpreters/QueryNormalizer.cpp | 115 --------------- dbms/src/Interpreters/QueryNormalizer.h | 26 +--- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 69 ++++----- .../TranslateQualifiedNamesVisitor.cpp | 134 +++++++++++++++++- .../TranslateQualifiedNamesVisitor.h | 25 +++- 7 files changed, 185 insertions(+), 191 deletions(-) diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index e7688903db8..0dba4f6a163 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -35,6 +35,7 @@ #include #include #include +#include namespace DB { diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index b49f02a14fa..930295a3b5a 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -338,9 +338,9 @@ ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast std::unordered_map aliases; std::vector tables = getDatabaseAndTables(*select_query, context.getCurrentDatabase()); - std::vector tables_with_columns; - TranslateQualifiedNamesVisitor::Data::setTablesOnly(tables, tables_with_columns); - TranslateQualifiedNamesVisitor::Data qn_visitor_data{{}, tables_with_columns}; + /// TODO: get tables from evaluateAsterisk instead of tablesOnly() to extract asterisks in general way + std::vector tables_with_columns = TranslateQualifiedNamesVisitor::Data::tablesOnly(tables); + TranslateQualifiedNamesVisitor::Data qn_visitor_data({}, tables_with_columns, false); TranslateQualifiedNamesVisitor(qn_visitor_data).visit(ast); QueryAliasesVisitor::Data query_aliases_data{aliases}; diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index 1a0f5bb5ac8..97d4d402f26 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -7,12 +7,10 @@ #include #include #include -#include #include #include #include #include -#include #include namespace DB @@ -91,13 +89,6 @@ void QueryNormalizer::visit(ASTFunction & node, const ASTPtr &, Data & data) /// and on all remote servers, function implementation will be same. if (endsWith(func_name, "Distinct") && func_name_lowercase == "countdistinct") func_name = data.settings.count_distinct_implementation; - - /// As special case, treat count(*) as count(), not as count(list of all columns). - if (func_name_lowercase == "count" && func_arguments->children.size() == 1 - && typeid_cast(func_arguments->children[0].get())) - { - func_arguments->children.clear(); - } } } @@ -138,84 +129,6 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) } } -/// Replace *, alias.*, database.table.* with a list of columns. -void QueryNormalizer::visit(ASTExpressionList & node, const ASTPtr &, Data & data) -{ - if (!data.tables_with_columns) - return; - - const auto & tables_with_columns = *data.tables_with_columns; - const auto & source_columns_set = data.source_columns_set; - - ASTs old_children; - if (data.processAsterisks()) - { - bool has_asterisk = false; - for (const auto & child : node.children) - { - if (typeid_cast(child.get()) || - typeid_cast(child.get())) - { - has_asterisk = true; - break; - } - } - - if (has_asterisk) - { - old_children.swap(node.children); - node.children.reserve(old_children.size()); - } - } - - for (const auto & child : old_children) - { - if (typeid_cast(child.get())) - { - bool first_table = true; - for (const auto & [table_name, table_columns] : tables_with_columns) - { - for (const auto & column_name : table_columns) - if (first_table || !data.join_using_columns.count(column_name)) - { - /// qualifed names for duplicates - if (!first_table && source_columns_set && source_columns_set->count(column_name)) - node.children.emplace_back(std::make_shared(table_name.getQualifiedNamePrefix() + column_name)); - else - node.children.emplace_back(std::make_shared(column_name)); - } - - first_table = false; - } - } - else if (const auto * qualified_asterisk = typeid_cast(child.get())) - { - DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]); - - bool first_table = true; - for (const auto & [table_name, table_columns] : tables_with_columns) - { - if (ident_db_and_name.satisfies(table_name, true)) - { - for (const auto & column_name : table_columns) - { - /// qualifed names for duplicates - if (!first_table && source_columns_set && source_columns_set->count(column_name)) - node.children.emplace_back(std::make_shared(table_name.getQualifiedNamePrefix() + column_name)); - else - node.children.emplace_back(std::make_shared(column_name)); - } - break; - } - - first_table = false; - } - } - else - node.children.emplace_back(child); - } -} - /// mark table identifiers as 'not columns' void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr &, Data &) { @@ -229,9 +142,6 @@ void QueryNormalizer::visit(ASTTablesInSelectQueryElement & node, const ASTPtr & /// special visitChildren() for ASTSelectQuery void QueryNormalizer::visit(ASTSelectQuery & select, const ASTPtr & ast, Data & data) { - if (auto join = select.join()) - extractJoinUsingColumns(join->table_join, data); - for (auto & child : ast->children) { if (typeid_cast(child.get()) || @@ -312,8 +222,6 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data) visit(*node, ast, data); if (auto * node = typeid_cast(ast.get())) visit(*node, ast, data); - if (auto * node = typeid_cast(ast.get())) - visit(*node, ast, data); if (auto * node = typeid_cast(ast.get())) visit(*node, ast, data); if (auto * node = typeid_cast(ast.get())) @@ -344,27 +252,4 @@ void QueryNormalizer::visit(ASTPtr & ast, Data & data) } } -/// 'select * from a join b using id' should result one 'id' column -void QueryNormalizer::extractJoinUsingColumns(const ASTPtr ast, Data & data) -{ - const auto & table_join = typeid_cast(*ast); - - if (table_join.using_expression_list) - { - auto & keys = typeid_cast(*table_join.using_expression_list); - for (const auto & key : keys.children) - if (auto opt_column = getIdentifierName(key)) - data.join_using_columns.insert(*opt_column); - else if (typeid_cast(key.get())) - data.join_using_columns.insert(key->getColumnName()); - else - { - String alias = key->tryGetAlias(); - if (alias.empty()) - throw Exception("Logical error: expected identifier or alias, got: " + key->getID(), ErrorCodes::LOGICAL_ERROR); - data.join_using_columns.insert(alias); - } - } -} - } diff --git a/dbms/src/Interpreters/QueryNormalizer.h b/dbms/src/Interpreters/QueryNormalizer.h index 517f9416959..6d6fea86e44 100644 --- a/dbms/src/Interpreters/QueryNormalizer.h +++ b/dbms/src/Interpreters/QueryNormalizer.h @@ -1,11 +1,9 @@ #pragma once -#include #include #include #include -#include #include namespace DB @@ -21,9 +19,9 @@ inline bool functionIsInOrGlobalInOperator(const String & name) return functionIsInOperator(name) || name == "globalIn" || name == "globalNotIn"; } +class ASTSelectQuery; class ASTFunction; class ASTIdentifier; -class ASTExpressionList; struct ASTTablesInSelectQueryElement; class Context; @@ -53,10 +51,6 @@ public: const Aliases & aliases; const ExtractedSettings settings; - const Context * context; - const NameSet * source_columns_set; - const std::vector * tables_with_columns; - std::unordered_set join_using_columns; /// tmp data size_t level; @@ -64,26 +58,11 @@ public: SetOfASTs current_asts; /// vertices in the current call stack of this method std::string current_alias; /// the alias referencing to the ancestor of ast (the deepest ancestor with aliases) - Data(const Aliases & aliases_, ExtractedSettings && settings_, const Context & context_, - const NameSet & source_columns_set, const std::vector & tables_with_columns_) - : aliases(aliases_) - , settings(settings_) - , context(&context_) - , source_columns_set(&source_columns_set) - , tables_with_columns(&tables_with_columns_) - , level(0) - {} - Data(const Aliases & aliases_, ExtractedSettings && settings_) : aliases(aliases_) , settings(settings_) - , context(nullptr) - , source_columns_set(nullptr) - , tables_with_columns(nullptr) , level(0) {} - - bool processAsterisks() const { return tables_with_columns && !tables_with_columns->empty(); } }; QueryNormalizer(Data & data) @@ -102,13 +81,10 @@ private: static void visit(ASTIdentifier &, ASTPtr &, Data &); static void visit(ASTFunction &, const ASTPtr &, Data &); - static void visit(ASTExpressionList &, const ASTPtr &, Data &); static void visit(ASTTablesInSelectQueryElement &, const ASTPtr &, Data &); static void visit(ASTSelectQuery &, const ASTPtr &, Data &); static void visitChildren(const ASTPtr &, Data & data); - - static void extractJoinUsingColumns(const ASTPtr ast, Data & data); }; } diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index b4dc9a31211..33cb2da0515 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -78,49 +78,36 @@ void collectSourceColumns(ASTSelectQuery * select_query, StoragePtr storage, Nam } } -/// Translate qualified names such as db.table.column, table.column, table_alias.column to unqualified names. -void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, const NameSet & source_columns, - const std::vector & tables_with_columns) +/// Translate qualified names such as db.table.column, table.column, table_alias.column to names' normal form. +/// Expand asterisks and qualified asterisks with column names. +/// There would be columns in normal form & column aliases after translation. Column & column alias would be normalized in QueryNormalizer. +void translateQualifiedNames(ASTPtr & query, ASTSelectQuery * select_query, const Context & context, SyntaxAnalyzerResult & result, + const Names & source_columns_list, const NameSet & source_columns_set) { - if (!select_query->tables || select_query->tables->children.empty()) - return; + std::vector tables_with_columns = getDatabaseAndTablesWithColumnNames(*select_query, context); + + if (tables_with_columns.empty()) + { + Names all_columns_name = source_columns_list; + + /// TODO: asterisk_left_columns_only probably does not work in some cases + if (!context.getSettingsRef().asterisk_left_columns_only) + { + auto columns_from_joined_table = result.analyzed_join.getColumnsFromJoinedTable(source_columns_set, context, select_query); + for (auto & column : columns_from_joined_table) + all_columns_name.emplace_back(column.name_and_type.name); + } + + tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, std::move(all_columns_name)); + } LogAST log; - TranslateQualifiedNamesVisitor::Data visitor_data{source_columns, tables_with_columns}; + TranslateQualifiedNamesVisitor::Data visitor_data(source_columns_set, tables_with_columns); TranslateQualifiedNamesVisitor visitor(visitor_data, log.stream()); visitor.visit(query); } -/// For star nodes(`*`), expand them to a list of all columns. For literal nodes, substitute aliases. -void normalizeTree( - ASTPtr & query, - SyntaxAnalyzerResult & result, - const Names & source_columns, - const NameSet & source_columns_set, - const Context & context, - const ASTSelectQuery * select_query, - std::vector & tables_with_columns) -{ - const auto & settings = context.getSettingsRef(); - Names all_columns_name = source_columns; - - if (!settings.asterisk_left_columns_only) - { - auto columns_from_joined_table = result.analyzed_join.getColumnsFromJoinedTable(source_columns_set, context, select_query); - for (auto & column : columns_from_joined_table) - all_columns_name.emplace_back(column.name_and_type.name); - } - - if (all_columns_name.empty()) - throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR); - - if (tables_with_columns.empty()) - tables_with_columns.emplace_back(DatabaseAndTableWithAlias{}, std::move(all_columns_name)); - - QueryNormalizer::Data normalizer_data(result.aliases, settings, context, source_columns_set, tables_with_columns); - QueryNormalizer(normalizer_data).visit(query); -} bool hasArrayJoin(const ASTPtr & ast) { if (const ASTFunction * function = typeid_cast(&*ast)) @@ -646,12 +633,10 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( if (source_columns_set.size() != source_columns_list.size()) throw Exception("Unexpected duplicates in source columns list.", ErrorCodes::LOGICAL_ERROR); - std::vector tables_with_columns; - if (select_query) { - tables_with_columns = getDatabaseAndTablesWithColumnNames(*select_query, context); - translateQualifiedNames(query, select_query, source_columns_set, tables_with_columns); + translateQualifiedNames(query, select_query, context, result, + (storage ? storage->getColumns().ordinary.getNames() : source_columns_list), source_columns_set); /// Depending on the user's profile, check for the execution rights /// distributed subqueries inside the IN or JOIN sections and process these subqueries. @@ -669,8 +654,10 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( } /// Common subexpression elimination. Rewrite rules. - normalizeTree(query, result, (storage ? storage->getColumns().ordinary.getNames() : source_columns_list), source_columns_set, - context, select_query, tables_with_columns); + { + QueryNormalizer::Data normalizer_data(result.aliases, context.getSettingsRef()); + QueryNormalizer(normalizer_data).visit(query); + } /// Remove unneeded columns according to 'required_result_columns'. /// Leave all selected columns in case of DISTINCT; columns that contain arrayJoin function inside. diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index f6e5ebe956a..febc9753366 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -1,3 +1,5 @@ +#include + #include #include @@ -5,10 +7,14 @@ #include #include +#include #include #include #include #include +#include +#include +#include namespace DB @@ -41,12 +47,14 @@ std::vector TranslateQualifiedNamesMatcher::visit(ASTPtr & ast, Data & { if (auto * t = typeid_cast(ast.get())) return visit(*t, ast, data); - if (auto * t = typeid_cast(ast.get())) - return visit(*t, ast, data); if (auto * t = typeid_cast(ast.get())) return visit(*t, ast, data); if (auto * t = typeid_cast(ast.get())) return visit(*t, ast, data); + if (auto * node = typeid_cast(ast.get())) + visit(*node, ast, data); + if (auto * node = typeid_cast(ast.get())) + visit(*node, ast, data); return {}; } @@ -73,6 +81,18 @@ std::vector TranslateQualifiedNamesMatcher::visit(ASTIdentifier & iden return {}; } +/// As special case, treat count(*) as count(), not as count(list of all columns). +void TranslateQualifiedNamesMatcher::visit(ASTFunction & node, const ASTPtr &, Data &) +{ + ASTPtr & func_arguments = node.arguments; + + String func_name_lowercase = Poco::toLower(node.name); + if (func_name_lowercase == "count" && + func_arguments->children.size() == 1 && + typeid_cast(func_arguments->children[0].get())) + func_arguments->children.clear(); +} + std::vector TranslateQualifiedNamesMatcher::visit(const ASTQualifiedAsterisk & , const ASTPtr & ast, Data & data) { if (ast->children.size() != 1) @@ -100,8 +120,11 @@ std::vector TranslateQualifiedNamesMatcher::visit(ASTTableJoin & join, return out; } -std::vector TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr & , Data &) +std::vector TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & select, const ASTPtr & , Data & data) { + if (auto join = select.join()) + extractJoinUsingColumns(join->table_join, data); + /// If the WHERE clause or HAVING consists of a single qualified column, the reference must be translated not only in children, /// but also in where_expression and having_expression. std::vector out; @@ -114,4 +137,109 @@ std::vector TranslateQualifiedNamesMatcher::visit(ASTSelectQuery & sel return out; } +/// Replace *, alias.*, database.table.* with a list of columns. +void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPtr &, Data & data) +{ + const auto & tables_with_columns = data.tables; + const auto & source_columns = data.source_columns; + + ASTs old_children; + if (data.processAsterisks()) + { + bool has_asterisk = false; + for (const auto & child : node.children) + { + if (typeid_cast(child.get())) + { + if (tables_with_columns.empty()) + throw Exception("An asterisk cannot be replaced with empty columns.", ErrorCodes::LOGICAL_ERROR); + has_asterisk = true; + break; + } + else if (auto qa = typeid_cast(child.get())) + { + visit(*qa, child, data); /// check if it's OK before rewrite + has_asterisk = true; + break; + } + } + + if (has_asterisk) + { + old_children.swap(node.children); + node.children.reserve(old_children.size()); + } + } + + for (const auto & child : old_children) + { + if (typeid_cast(child.get())) + { + bool first_table = true; + for (const auto & [table_name, table_columns] : tables_with_columns) + { + for (const auto & column_name : table_columns) + if (first_table || !data.join_using_columns.count(column_name)) + { + /// qualifed names for duplicates + if (!first_table && source_columns.count(column_name)) + node.children.emplace_back(std::make_shared(table_name.getQualifiedNamePrefix() + column_name)); + else + node.children.emplace_back(std::make_shared(column_name)); + } + + first_table = false; + } + } + else if (const auto * qualified_asterisk = typeid_cast(child.get())) + { + DatabaseAndTableWithAlias ident_db_and_name(qualified_asterisk->children[0]); + + bool first_table = true; + for (const auto & [table_name, table_columns] : tables_with_columns) + { + if (ident_db_and_name.satisfies(table_name, true)) + { + for (const auto & column_name : table_columns) + { + /// qualifed names for duplicates + if (!first_table && source_columns.count(column_name)) + node.children.emplace_back(std::make_shared(table_name.getQualifiedNamePrefix() + column_name)); + else + node.children.emplace_back(std::make_shared(column_name)); + } + break; + } + + first_table = false; + } + } + else + node.children.emplace_back(child); + } +} + +/// 'select * from a join b using id' should result one 'id' column +void TranslateQualifiedNamesMatcher::extractJoinUsingColumns(const ASTPtr ast, Data & data) +{ + const auto & table_join = typeid_cast(*ast); + + if (table_join.using_expression_list) + { + auto & keys = typeid_cast(*table_join.using_expression_list); + for (const auto & key : keys.children) + if (auto opt_column = getIdentifierName(key)) + data.join_using_columns.insert(*opt_column); + else if (typeid_cast(key.get())) + data.join_using_columns.insert(key->getColumnName()); + else + { + String alias = key->tryGetAlias(); + if (alias.empty()) + throw Exception("Logical error: expected identifier or alias, got: " + key->getID(), ErrorCodes::LOGICAL_ERROR); + data.join_using_columns.insert(alias); + } + } +} + } diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h index bee5e7022f4..15dd6e5192f 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -13,6 +13,8 @@ class ASTIdentifier; class ASTQualifiedAsterisk; struct ASTTableJoin; class ASTSelectQuery; +class ASTExpressionList; +class ASTFunction; /// Visit one node for names qualification. @sa InDepthNodeVisitor. class TranslateQualifiedNamesMatcher @@ -22,15 +24,26 @@ public: { const NameSet & source_columns; const std::vector & tables; + std::unordered_set join_using_columns; + bool has_columns; - static void setTablesOnly(const std::vector & tables, - std::vector & tables_with_columns) + Data(const NameSet & source_columns_, const std::vector & tables_, bool has_columns_ = true) + : source_columns(source_columns_) + , tables(tables_) + , has_columns(has_columns_) + {} + + static std::vector tablesOnly(const std::vector & tables) { - tables_with_columns.clear(); + std::vector tables_with_columns; tables_with_columns.reserve(tables.size()); + for (const auto & table : tables) tables_with_columns.emplace_back(TableWithColumnNames{table, {}}); + return tables_with_columns; } + + bool processAsterisks() const { return !tables.empty() && has_columns; } }; static constexpr const char * label = "TranslateQualifiedNames"; @@ -43,10 +56,14 @@ private: static std::vector visit(const ASTQualifiedAsterisk & node, const ASTPtr & ast, Data &); static std::vector visit(ASTTableJoin & node, const ASTPtr & ast, Data &); static std::vector visit(ASTSelectQuery & node, const ASTPtr & ast, Data &); + static void visit(ASTExpressionList &, const ASTPtr &, Data &); + static void visit(ASTFunction &, const ASTPtr &, Data &); + + static void extractJoinUsingColumns(const ASTPtr ast, Data & data); }; /// Visits AST for names qualification. -/// It finds columns (general identifiers and asterisks) and translate their names according to tables' names. +/// It finds columns and translate their names to the normal form. Expand asterisks and qualified asterisks with column names. using TranslateQualifiedNamesVisitor = InDepthNodeVisitor; } From c35439a5f013c997e0f7cb8171dd0c3d9a4752c8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 11 Feb 2019 16:11:52 +0300 Subject: [PATCH 03/40] Updated FieldVisitor. --- .../AggregateFunctionFactory.cpp | 7 +++- .../AggregateFunctions/IAggregateFunction.h | 15 ++++++++ dbms/src/Columns/ColumnAggregateFunction.cpp | 27 ++++++++++--- dbms/src/Columns/ColumnAggregateFunction.h | 2 + dbms/src/Common/FieldVisitors.cpp | 14 +++++-- dbms/src/Common/FieldVisitors.h | 19 ++++++---- dbms/src/Core/Field.cpp | 12 ++++-- dbms/src/Core/Field.h | 38 ++++++++++++++++++- dbms/src/DataTypes/FieldToDataType.cpp | 6 +++ dbms/src/DataTypes/FieldToDataType.h | 1 + dbms/src/Interpreters/convertFieldToType.cpp | 15 ++++++++ 11 files changed, 132 insertions(+), 24 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 6aeaaef2bfa..932d6615385 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -86,12 +86,17 @@ AggregateFunctionPtr AggregateFunctionFactory::get( [](const auto & type) { return type->onlyNull(); })) nested_function = getImpl(name, nested_types, parameters, recursion_level); - return combinator->transformAggregateFunction(nested_function, argument_types, parameters); + auto res = combinator->transformAggregateFunction(nested_function, type_without_low_cardinality, parameters); + res->setArguments(type_without_low_cardinality, parameters); + return res; } auto res = getImpl(name, type_without_low_cardinality, parameters, recursion_level); if (!res) throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR); + + res->setArguments(type_without_low_cardinality, parameters); + return res; } diff --git a/dbms/src/AggregateFunctions/IAggregateFunction.h b/dbms/src/AggregateFunctions/IAggregateFunction.h index 85de0e0c7ff..f5def066058 100644 --- a/dbms/src/AggregateFunctions/IAggregateFunction.h +++ b/dbms/src/AggregateFunctions/IAggregateFunction.h @@ -108,6 +108,21 @@ public: * const char * getHeaderFilePath() const override { return __FILE__; } */ virtual const char * getHeaderFilePath() const = 0; + + const DataTypes & getArgumentTypes() const { return argument_types; } + const Array & getParameters() const { return parameters; } + +private: + DataTypes argument_types; + Array parameters; + + friend class AggregateFunctionFactory; + + void setArguments(DataTypes argument_types_, Array parameters_) + { + argument_types = std::move(argument_types_); + parameters = std::move(parameters_); + } }; diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index 65b168e35a9..23abee39530 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include #include #include #include @@ -258,11 +260,17 @@ MutableColumnPtr ColumnAggregateFunction::cloneEmpty() const return create(func, Arenas(1, std::make_shared())); } +String ColumnAggregateFunction::getTypeString() const +{ + return DataTypeAggregateFunction(func, func->getArgumentTypes(), func->getParameters()).getName(); +} + Field ColumnAggregateFunction::operator[](size_t n) const { Field field = AggregateFunctionStateData(); + field.get().name = getTypeString(); { - WriteBufferFromString buffer(field.get().toUnderType()); + WriteBufferFromString buffer(field.get().data); func->serialize(data[n], buffer); } return field; @@ -270,9 +278,10 @@ Field ColumnAggregateFunction::operator[](size_t n) const void ColumnAggregateFunction::get(size_t n, Field & res) const { - res = String(); + res = AggregateFunctionStateData(); + res.get().name = getTypeString(); { - WriteBufferFromString buffer(res.get().toUnderType()); + WriteBufferFromString buffer(res.get().data); func->serialize(data[n], buffer); } } @@ -337,13 +346,18 @@ static void pushBackAndCreateState(ColumnAggregateFunction::Container & data, Ar } } - void ColumnAggregateFunction::insert(const Field & x) { + String type_string = getTypeString(); + auto & field_name = x.get().name; + if (type_string != field_name) + throw Exception("Cannot insert filed with type " + field_name + " into column with type " + type_string, + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ensureOwnership(); Arena & arena = createOrGetArena(); pushBackAndCreateState(data, arena, func.get()); - ReadBufferFromString read_buffer(x.get().toUnderType()); + ReadBufferFromString read_buffer(x.get().data); func->deserialize(data.back(), read_buffer, &arena); } @@ -466,11 +480,12 @@ void ColumnAggregateFunction::getExtremes(Field & min, Field & max) const AggregateDataPtr place = place_buffer.data(); AggregateFunctionStateData serialized; + serialized.name = getTypeString(); func->create(place); try { - WriteBufferFromString buffer(serialized.toUnderType()); + WriteBufferFromString buffer(serialized.data); func->serialize(place, buffer); } catch (...) diff --git a/dbms/src/Columns/ColumnAggregateFunction.h b/dbms/src/Columns/ColumnAggregateFunction.h index e44689d992e..e090cb8c110 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.h +++ b/dbms/src/Columns/ColumnAggregateFunction.h @@ -94,6 +94,8 @@ private: { } + String getTypeString() const; + public: ~ColumnAggregateFunction() override; diff --git a/dbms/src/Common/FieldVisitors.cpp b/dbms/src/Common/FieldVisitors.cpp index e10095fed90..f77977b3eed 100644 --- a/dbms/src/Common/FieldVisitors.cpp +++ b/dbms/src/Common/FieldVisitors.cpp @@ -92,7 +92,8 @@ String FieldVisitorDump::operator() (const Tuple & x_def) const String FieldVisitorDump::operator() (const AggregateFunctionStateData & x) const { WriteBufferFromOwnString wb; - writeQuoted(x, wb); + writeQuoted(x.name, wb); + writeQuoted(x.data, wb); return wb.str(); } @@ -127,7 +128,10 @@ String FieldVisitorToString::operator() (const DecimalField & x) cons String FieldVisitorToString::operator() (const DecimalField & x) const { return formatQuoted(x); } String FieldVisitorToString::operator() (const DecimalField & x) const { return formatQuoted(x); } String FieldVisitorToString::operator() (const UInt128 & x) const { return formatQuoted(UUID(x)); } -String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const { return formatQuoted(x); } +String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const +{ + return "(" + formatQuoted(x.name) + ")" + formatQuoted(x.data); +} String FieldVisitorToString::operator() (const Array & x) const { @@ -242,8 +246,10 @@ void FieldVisitorHash::operator() (const AggregateFunctionStateData & x) const { UInt8 type = Field::Types::AggregateFunctionState; hash.update(type); - hash.update(x.toUnderType().size()); - hash.update(x.toUnderType().data(), x.toUnderType().size()); + hash.update(x.name.size()); + hash.update(x.name.data(), x.name.size()); + hash.update(x.data.size()); + hash.update(x.data.data(), x.data.size()); } diff --git a/dbms/src/Common/FieldVisitors.h b/dbms/src/Common/FieldVisitors.h index 30aa700f80d..5575c607b3b 100644 --- a/dbms/src/Common/FieldVisitors.h +++ b/dbms/src/Common/FieldVisitors.h @@ -49,6 +49,7 @@ typename std::decay_t::ResultType applyVisitor(Visitor && visitor, F && case Field::Types::Decimal32: return visitor(field.template get>()); case Field::Types::Decimal64: return visitor(field.template get>()); case Field::Types::Decimal128: return visitor(field.template get>()); + case Field::Types::AggregateFunctionState: return visitor(field.template get()); default: throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); @@ -72,6 +73,7 @@ static typename std::decay_t::ResultType applyBinaryVisitorImpl(Visitor case Field::Types::Decimal32: return visitor(field1, field2.template get>()); case Field::Types::Decimal64: return visitor(field1, field2.template get>()); case Field::Types::Decimal128: return visitor(field1, field2.template get>()); + case Field::Types::AggregateFunctionState: return visitor(field1, field2.template get()); default: throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); @@ -116,6 +118,9 @@ typename std::decay_t::ResultType applyVisitor(Visitor && visitor, F1 & case Field::Types::Decimal128: return applyBinaryVisitorImpl( std::forward(visitor), field1.template get>(), std::forward(field2)); + case Field::Types::AggregateFunctionState: + return applyBinaryVisitorImpl( + std::forward(visitor), field1.template get(), std::forward(field2)); default: throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); @@ -206,7 +211,7 @@ public: T operator() (const AggregateFunctionStateData &) const { - throw Exception("Cannot convert String to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE); + throw Exception("Cannot convert AggregateFunctionStateData to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE); } }; @@ -254,6 +259,7 @@ public: bool operator() (const UInt64 & l, const String & r) const { return cantCompare(l, r); } bool operator() (const UInt64 & l, const Array & r) const { return cantCompare(l, r); } bool operator() (const UInt64 & l, const Tuple & r) const { return cantCompare(l, r); } + bool operator() (const UInt64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } bool operator() (const Int64 & l, const Null & r) const { return cantCompare(l, r); } bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); } @@ -263,6 +269,7 @@ public: bool operator() (const Int64 & l, const String & r) const { return cantCompare(l, r); } bool operator() (const Int64 & l, const Array & r) const { return cantCompare(l, r); } bool operator() (const Int64 & l, const Tuple & r) const { return cantCompare(l, r); } + bool operator() (const Int64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } bool operator() (const Float64 & l, const Null & r) const { return cantCompare(l, r); } bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); } @@ -272,6 +279,7 @@ public: bool operator() (const Float64 & l, const String & r) const { return cantCompare(l, r); } bool operator() (const Float64 & l, const Array & r) const { return cantCompare(l, r); } bool operator() (const Float64 & l, const Tuple & r) const { return cantCompare(l, r); } + bool operator() (const Float64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } template bool operator() (const Null &, const T &) const @@ -334,8 +342,6 @@ public: { if constexpr (std::is_same_v) return l == r; - if constexpr (std::is_same_v) - return stringToUUID(l.toUnderType()) == r; return cantCompare(l, r); } @@ -362,6 +368,7 @@ public: bool operator() (const UInt64 & l, const String & r) const { return cantCompare(l, r); } bool operator() (const UInt64 & l, const Array & r) const { return cantCompare(l, r); } bool operator() (const UInt64 & l, const Tuple & r) const { return cantCompare(l, r); } + bool operator() (const UInt64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } bool operator() (const Int64 & l, const Null & r) const { return cantCompare(l, r); } bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); } @@ -371,6 +378,7 @@ public: bool operator() (const Int64 & l, const String & r) const { return cantCompare(l, r); } bool operator() (const Int64 & l, const Array & r) const { return cantCompare(l, r); } bool operator() (const Int64 & l, const Tuple & r) const { return cantCompare(l, r); } + bool operator() (const Int64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } bool operator() (const Float64 & l, const Null & r) const { return cantCompare(l, r); } bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); } @@ -380,6 +388,7 @@ public: bool operator() (const Float64 & l, const String & r) const { return cantCompare(l, r); } bool operator() (const Float64 & l, const Array & r) const { return cantCompare(l, r); } bool operator() (const Float64 & l, const Tuple & r) const { return cantCompare(l, r); } + bool operator() (const Float64 & l, const AggregateFunctionStateData & r) const { return cantCompare(l, r); } template bool operator() (const Null &, const T &) const @@ -440,10 +449,6 @@ public: template bool operator() (const AggregateFunctionStateData & l, const T & r) const { - if constexpr (std::is_same_v) - return l < r; - if constexpr (std::is_same_v) - return stringToUUID(l.toUnderType()) < r; return cantCompare(l, r); } diff --git a/dbms/src/Core/Field.cpp b/dbms/src/Core/Field.cpp index 7411d17efc1..fbe436a76fe 100644 --- a/dbms/src/Core/Field.cpp +++ b/dbms/src/Core/Field.cpp @@ -78,7 +78,8 @@ namespace DB case Field::Types::AggregateFunctionState: { AggregateFunctionStateData value; - DB::readStringBinary(value, buf); + DB::readStringBinary(value.name, buf); + DB::readStringBinary(value.data, buf); x.push_back(value); break; } @@ -137,7 +138,8 @@ namespace DB } case Field::Types::AggregateFunctionState: { - DB::writeStringBinary(get(*it), buf); + DB::writeStringBinary(it->get().name, buf); + DB::writeStringBinary(it->get().data, buf); break; } } @@ -224,7 +226,8 @@ namespace DB case Field::Types::AggregateFunctionState: { AggregateFunctionStateData value; - DB::readStringBinary(value, buf); + DB::readStringBinary(value.name, buf); + DB::readStringBinary(value.data, buf); x.push_back(value); break; } @@ -283,7 +286,8 @@ namespace DB } case Field::Types::AggregateFunctionState: { - DB::writeStringBinary(get(*it), buf); + DB::writeStringBinary(it->get().name, buf); + DB::writeStringBinary(it->get().data, buf); break; } } diff --git a/dbms/src/Core/Field.h b/dbms/src/Core/Field.h index 583b3303c56..7afa1395710 100644 --- a/dbms/src/Core/Field.h +++ b/dbms/src/Core/Field.h @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int BAD_GET; extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } class Field; @@ -30,8 +31,40 @@ using Array = std::vector; using TupleBackend = std::vector; STRONG_TYPEDEF(TupleBackend, Tuple) /// Array and Tuple are different types with equal representation inside Field. -using AggregateFunctionStateDataBackend = String; -STRONG_TYPEDEF(AggregateFunctionStateDataBackend, AggregateFunctionStateData) +struct AggregateFunctionStateData +{ + String name; /// Name with arguments. + String data; + + bool operator < (const AggregateFunctionStateData &) const + { + throw Exception("Operator < is not implemented for AggregateFunctionStateData.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + bool operator <= (const AggregateFunctionStateData &) const + { + throw Exception("Operator <= is not implemented for AggregateFunctionStateData.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + bool operator > (const AggregateFunctionStateData &) const + { + throw Exception("Operator > is not implemented for AggregateFunctionStateData.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + bool operator >= (const AggregateFunctionStateData &) const + { + throw Exception("Operator >= is not implemented for AggregateFunctionStateData.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + bool operator == (const AggregateFunctionStateData & rhs) const + { + if (name != rhs.name) + throw Exception("Comparing aggregate functions with different types: " + name + " and " + rhs.name, + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return data == rhs.data; + } +}; template bool decimalEqual(T x, T y, UInt32 x_scale, UInt32 y_scale); template bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale); @@ -587,6 +620,7 @@ T safeGet(Field & field) template <> struct TypeName { static std::string get() { return "Array"; } }; template <> struct TypeName { static std::string get() { return "Tuple"; } }; +template <> struct TypeName { static std::string get() { return "AggregateFunctionState"; } }; template struct NearestFieldTypeImpl; diff --git a/dbms/src/DataTypes/FieldToDataType.cpp b/dbms/src/DataTypes/FieldToDataType.cpp index 18fa0e97b88..70fab533838 100644 --- a/dbms/src/DataTypes/FieldToDataType.cpp +++ b/dbms/src/DataTypes/FieldToDataType.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -104,5 +105,10 @@ DataTypePtr FieldToDataType::operator() (const Tuple & x) const return std::make_shared(element_types); } +DataTypePtr FieldToDataType::operator() (const AggregateFunctionStateData & x) const +{ + auto & name = static_cast(x).name; + return DataTypeFactory::instance().get(name); +} } diff --git a/dbms/src/DataTypes/FieldToDataType.h b/dbms/src/DataTypes/FieldToDataType.h index dc103e24641..1edcdf3c11d 100644 --- a/dbms/src/DataTypes/FieldToDataType.h +++ b/dbms/src/DataTypes/FieldToDataType.h @@ -28,6 +28,7 @@ public: DataTypePtr operator() (const DecimalField & x) const; DataTypePtr operator() (const DecimalField & x) const; DataTypePtr operator() (const DecimalField & x) const; + DataTypePtr operator() (const AggregateFunctionStateData & x) const; }; } diff --git a/dbms/src/Interpreters/convertFieldToType.cpp b/dbms/src/Interpreters/convertFieldToType.cpp index 892a5967719..378cd7db6b8 100644 --- a/dbms/src/Interpreters/convertFieldToType.cpp +++ b/dbms/src/Interpreters/convertFieldToType.cpp @@ -22,6 +22,7 @@ #include #include +#include namespace DB @@ -248,6 +249,18 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID return res; } } + else if (const DataTypeAggregateFunction * agg_func_type = typeid_cast(&type)) + { + if (src.getType() != Field::Types::AggregateFunctionState) + throw Exception(String("Cannot convert ") + src.getTypeName() + " to " + agg_func_type->getName(), + ErrorCodes::TYPE_MISMATCH); + + auto & name = src.get().name; + if (agg_func_type->getName() != name) + throw Exception("Cannot convert " + name + " to " + agg_func_type->getName(), ErrorCodes::TYPE_MISMATCH); + + return src; + } if (src.getType() == Field::Types::String) { @@ -257,6 +270,8 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID return (*col)[0]; } + + // TODO (nemkov): should we attempt to parse value using or `type.deserializeAsTextEscaped()` type.deserializeAsTextEscaped() ? throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: " + Field::Types::toString(src.getType()), ErrorCodes::TYPE_MISMATCH); From d4ba5432b30ba8a8b6f091718f62531da6afb6bd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 11 Feb 2019 18:45:14 +0300 Subject: [PATCH 04/40] Added test. --- .../00905_field_with_aggregate_function_state.reference | 3 +++ .../0_stateless/00905_field_with_aggregate_function_state.sql | 4 ++++ 2 files changed, 7 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00905_field_with_aggregate_function_state.reference create mode 100644 dbms/tests/queries/0_stateless/00905_field_with_aggregate_function_state.sql diff --git a/dbms/tests/queries/0_stateless/00905_field_with_aggregate_function_state.reference b/dbms/tests/queries/0_stateless/00905_field_with_aggregate_function_state.reference new file mode 100644 index 00000000000..6d6b7b02313 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00905_field_with_aggregate_function_state.reference @@ -0,0 +1,3 @@ +1 +45 +4.5 diff --git a/dbms/tests/queries/0_stateless/00905_field_with_aggregate_function_state.sql b/dbms/tests/queries/0_stateless/00905_field_with_aggregate_function_state.sql new file mode 100644 index 00000000000..b0470ac9992 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00905_field_with_aggregate_function_state.sql @@ -0,0 +1,4 @@ +with (select sumState(1)) as s select sumMerge(s); +with (select sumState(number) from (select * from system.numbers limit 10)) as s select sumMerge(s); +with (select quantileState(0.5)(number) from (select * from system.numbers limit 10)) as s select quantileMerge(s); + From 0a6f75a1b67c41de40238f73a306a2471e8699f8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 11 Feb 2019 18:50:52 +0300 Subject: [PATCH 05/40] Allow to execute subquery with scalar aggregate function state. --- .../Interpreters/ExecuteScalarSubqueriesVisitor.cpp | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index b09ad5fb36e..cec1fa90962 100644 --- a/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/dbms/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -109,11 +109,6 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr size_t columns = block.columns(); if (columns == 1) { - if (typeid_cast(block.safeGetByPosition(0).type.get())) - { - throw Exception("Scalar subquery can't return an aggregate function state", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); - } - auto lit = std::make_unique((*block.safeGetByPosition(0).column)[0]); lit->alias = subquery.alias; lit->prefer_alias_to_column_name = subquery.prefer_alias_to_column_name; @@ -132,11 +127,6 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr exp_list->children.resize(columns); for (size_t i = 0; i < columns; ++i) { - if (typeid_cast(block.safeGetByPosition(i).type.get())) - { - throw Exception("Scalar subquery can't return an aggregate function state", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); - } - exp_list->children[i] = addTypeConversion( std::make_unique((*block.safeGetByPosition(i).column)[0]), block.safeGetByPosition(i).type->getName()); From 958b538a0135714543fda7ac6d7a372e7f814119 Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 11 Feb 2019 21:26:40 +0300 Subject: [PATCH 06/40] minor improvement --- dbms/src/Interpreters/QueryAliasesVisitor.cpp | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.cpp b/dbms/src/Interpreters/QueryAliasesVisitor.cpp index 3c10f7da1b5..3c17f9d4684 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.cpp +++ b/dbms/src/Interpreters/QueryAliasesVisitor.cpp @@ -56,14 +56,16 @@ std::vector QueryAliasesMatcher::visit(const ASTArrayJoin &, const AST { visitOther(ast, data); - /// @warning It breaks botom-to-top order (childs processed after node here), could lead to some effects. - /// It's possible to add ast back to result vec to save order. It will need two phase ASTArrayJoin visit (setting phase in data). - std::vector out; + std::vector grand_children; for (auto & child1 : ast->children) for (auto & child2 : child1->children) for (auto & child3 : child2->children) - out.push_back(&child3); - return out; + grand_children.push_back(child3); + + /// create own visitor to run bottom to top + for (auto & child : grand_children) + QueryAliasesVisitor(data).visit(child); + return {}; } /// set unique aliases for all subqueries. this is needed, because: From 805866e6b3cd38e8b1e4db3155e7aa4520e58087 Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 11 Feb 2019 22:14:57 +0300 Subject: [PATCH 07/40] hotfix for wrong aliases issue-4110 --- dbms/src/Interpreters/IdentifierSemantic.cpp | 9 ++++++++- dbms/src/Interpreters/IdentifierSemantic.h | 2 ++ dbms/src/Interpreters/QueryNormalizer.cpp | 2 +- .../queries/0_stateless/00818_alias_bug_4110.reference | 5 +++++ dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql | 5 +++++ 5 files changed, 21 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference create mode 100644 dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql diff --git a/dbms/src/Interpreters/IdentifierSemantic.cpp b/dbms/src/Interpreters/IdentifierSemantic.cpp index 13a9c49c3e0..41dc565811a 100644 --- a/dbms/src/Interpreters/IdentifierSemantic.cpp +++ b/dbms/src/Interpreters/IdentifierSemantic.cpp @@ -37,12 +37,17 @@ std::optional IdentifierSemantic::getTableName(const ASTPtr & ast) return {}; } - void IdentifierSemantic::setNeedLongName(ASTIdentifier & identifier, bool value) { identifier.semantic->need_long_name = value; } +bool IdentifierSemantic::canBeAlias(const ASTIdentifier & identifier) +{ + return identifier.semantic->can_be_alias; +} + + std::pair IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier) { if (identifier.name_parts.size() > 2) @@ -108,6 +113,8 @@ void IdentifierSemantic::setColumnNormalName(ASTIdentifier & identifier, const D size_t match = IdentifierSemantic::canReferColumnToTable(identifier, db_and_table); setColumnShortName(identifier, match); + if (match) + identifier.semantic->can_be_alias = false; if (identifier.semantic->need_long_name) { diff --git a/dbms/src/Interpreters/IdentifierSemantic.h b/dbms/src/Interpreters/IdentifierSemantic.h index be721627e1a..4318142c146 100644 --- a/dbms/src/Interpreters/IdentifierSemantic.h +++ b/dbms/src/Interpreters/IdentifierSemantic.h @@ -10,6 +10,7 @@ struct IdentifierSemanticImpl { bool special = false; bool need_long_name = false; + bool can_be_alias = true; }; /// Static calss to manipulate IdentifierSemanticImpl via ASTIdentifier @@ -28,6 +29,7 @@ struct IdentifierSemantic static String columnNormalName(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static void setColumnNormalName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static void setNeedLongName(ASTIdentifier & identifier, bool); /// if set setColumnNormalName makes qualified name + static bool canBeAlias(const ASTIdentifier & identifier); private: static bool doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table); diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index 97d4d402f26..936c99afc51 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -102,7 +102,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column"). auto it_alias = data.aliases.find(node.name); - if (it_alias != data.aliases.end() && current_alias != node.name) + if (IdentifierSemantic::canBeAlias(node) && it_alias != data.aliases.end() && current_alias != node.name) { auto & alias_node = it_alias->second; diff --git a/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference new file mode 100644 index 00000000000..204377075a9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference @@ -0,0 +1,5 @@ +10 11 +11 10 +11 11 +11 12 +12 11 diff --git a/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql new file mode 100644 index 00000000000..b1cfa3780a2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql @@ -0,0 +1,5 @@ +select s.a as a, s.a + 1 as b from (select 10 as a) s; +select s.a + 1 as a, s.a as b from (select 10 as a) s; +select s.a + 1 as a, s.a + 1 as b from (select 10 as a) s; +select s.a + 1 as b, s.a + 2 as a from (select 10 as a) s; +select s.a + 2 as b, s.a + 1 as a from (select 10 as a) s; From 2b8b342ccd3b6030d02ba16884112b812f2f5f3f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 11 Feb 2019 22:26:32 +0300 Subject: [PATCH 08/40] Update IAggregateFunction interface. --- .../AggregateFunctionArgMinMax.h | 7 ++-- .../AggregateFunctionArray.h | 3 +- .../AggregateFunctions/AggregateFunctionAvg.h | 3 +- .../AggregateFunctionBitwise.cpp | 2 +- .../AggregateFunctionBitwise.h | 3 ++ .../AggregateFunctionBoundingRatio.h | 1 + .../AggregateFunctionCount.cpp | 4 +- .../AggregateFunctionCount.h | 8 +++- .../AggregateFunctionEntropy.cpp | 4 +- .../AggregateFunctionEntropy.h | 4 +- .../AggregateFunctionFactory.cpp | 7 +--- .../AggregateFunctionForEach.h | 3 +- .../AggregateFunctionGroupArray.h | 5 ++- .../AggregateFunctionGroupArrayInsertAt.cpp | 4 ++ .../AggregateFunctionGroupArrayInsertAt.h | 9 ++--- .../AggregateFunctionGroupUniqArray.cpp | 8 +++- .../AggregateFunctionGroupUniqArray.h | 8 +++- .../AggregateFunctionHistogram.cpp | 2 +- .../AggregateFunctionHistogram.h | 5 ++- .../AggregateFunctions/AggregateFunctionIf.h | 3 +- .../AggregateFunctionMaxIntersections.h | 2 +- .../AggregateFunctionMerge.cpp | 2 +- .../AggregateFunctionMerge.h | 9 +++-- .../AggregateFunctionMinMaxAny.h | 6 ++- .../AggregateFunctionNothing.h | 3 ++ .../AggregateFunctionNull.cpp | 16 ++++---- .../AggregateFunctionNull.h | 12 +++--- .../AggregateFunctionQuantile.h | 5 ++- .../AggregateFunctionRetention.h | 1 + .../AggregateFunctionSequenceMatch.cpp | 4 +- .../AggregateFunctionSequenceMatch.h | 11 +++++- .../AggregateFunctionState.h | 3 +- .../AggregateFunctionStatistics.cpp | 4 +- .../AggregateFunctionStatistics.h | 7 ++++ .../AggregateFunctionStatisticsSimple.h | 10 +++-- .../AggregateFunctionSum.cpp | 4 +- .../AggregateFunctions/AggregateFunctionSum.h | 10 +++-- .../AggregateFunctionSumMap.cpp | 4 +- .../AggregateFunctionSumMap.h | 17 ++++++--- .../AggregateFunctionTopK.cpp | 14 +++---- .../AggregateFunctionTopK.h | 13 ++++--- .../AggregateFunctionUniq.cpp | 18 ++++----- .../AggregateFunctionUniq.h | 4 ++ .../AggregateFunctionUniqCombined.cpp | 38 +++++++++---------- .../AggregateFunctionUniqCombined.h | 7 +++- .../AggregateFunctionUniqUpTo.cpp | 18 ++++----- .../AggregateFunctionUniqUpTo.h | 10 +++-- .../AggregateFunctionWindowFunnel.h | 1 + .../AggregateFunctionsStatisticsSimple.cpp | 6 +-- .../AggregateFunctions/IAggregateFunction.h | 19 +++++----- 50 files changed, 220 insertions(+), 151 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h b/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h index 9a232e2e77d..42649be78fd 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h @@ -31,12 +31,13 @@ template class AggregateFunctionArgMinMax final : public IAggregateFunctionDataHelper> { private: - DataTypePtr type_res; - DataTypePtr type_val; + const DataTypePtr & type_res; + const DataTypePtr & type_val; public: AggregateFunctionArgMinMax(const DataTypePtr & type_res, const DataTypePtr & type_val) - : type_res(type_res), type_val(type_val) + : IAggregateFunctionDataHelper>({type_res, type_val}, {}), + type_res(argument_types[0]), type_val(argument_types[1]) { if (!type_val->isComparable()) throw Exception("Illegal type " + type_val->getName() + " of second argument of aggregate function " + getName() diff --git a/dbms/src/AggregateFunctions/AggregateFunctionArray.h b/dbms/src/AggregateFunctions/AggregateFunctionArray.h index 5dfebf13d52..08fa7c13bc3 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionArray.h @@ -28,7 +28,8 @@ private: public: AggregateFunctionArray(AggregateFunctionPtr nested_, const DataTypes & arguments) - : nested_func(nested_), num_arguments(arguments.size()) + : IAggregateFunctionHelper(arguments, {}) + , nested_func(nested_), num_arguments(arguments.size()) { for (const auto & type : arguments) if (!isArray(type)) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionAvg.h b/dbms/src/AggregateFunctions/AggregateFunctionAvg.h index 53b42c42c9a..98604f76742 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionAvg.h @@ -55,7 +55,8 @@ public: /// ctor for Decimals AggregateFunctionAvg(const IDataType & data_type) - : scale(getDecimalScale(data_type)) + : IAggregateFunctionDataHelper>({data_type}, {}) + , scale(getDecimalScale(data_type)) {} String getName() const override { return "avg"; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBitwise.cpp b/dbms/src/AggregateFunctions/AggregateFunctionBitwise.cpp index 8c188bcbb8e..e92e1917bd5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBitwise.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionBitwise.cpp @@ -21,7 +21,7 @@ AggregateFunctionPtr createAggregateFunctionBitwise(const std::string & name, co + " is illegal, because it cannot be used in bitwise operations", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - AggregateFunctionPtr res(createWithUnsignedIntegerType(*argument_types[0])); + AggregateFunctionPtr res(createWithUnsignedIntegerType(*argument_types[0], argument_types[0])); if (!res) throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h b/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h index 6d33f010bd0..2788fdccd51 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionBitwise.h @@ -43,6 +43,9 @@ template class AggregateFunctionBitwise final : public IAggregateFunctionDataHelper> { public: + AggregateFunctionBitwise(const DataTypePtr & type) + : IAggregateFunctionDataHelper>({type}, {}) {} + String getName() const override { return Data::name(); } DataTypePtr getReturnType() const override diff --git a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index 40b13acbbaa..5966993dc65 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -111,6 +111,7 @@ public: } AggregateFunctionBoundingRatio(const DataTypes & arguments) + : IAggregateFunctionDataHelper(arguments, {}) { const auto x_arg = arguments.at(0).get(); const auto y_arg = arguments.at(0).get(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionCount.cpp b/dbms/src/AggregateFunctions/AggregateFunctionCount.cpp index 1df424ecbf2..02dc796a4cf 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionCount.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionCount.cpp @@ -9,12 +9,12 @@ namespace DB namespace { -AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & /*argument_types*/, const Array & parameters) +AggregateFunctionPtr createAggregateFunctionCount(const std::string & name, const DataTypes & argument_types, const Array & parameters) { assertNoParameters(name, parameters); /// 'count' accept any number of arguments and (in this case of non-Nullable types) simply ignore them. - return std::make_shared(); + return std::make_shared(argument_types); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionCount.h b/dbms/src/AggregateFunctions/AggregateFunctionCount.h index f9a1dcb45e2..82958a95fd2 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionCount.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionCount.h @@ -28,6 +28,8 @@ namespace ErrorCodes class AggregateFunctionCount final : public IAggregateFunctionDataHelper { public: + AggregateFunctionCount(const DataTypes & argument_types) : IAggregateFunctionDataHelper(argument_types, {}) {} + String getName() const override { return "count"; } DataTypePtr getReturnType() const override @@ -74,7 +76,8 @@ public: class AggregateFunctionCountNotNullUnary final : public IAggregateFunctionDataHelper { public: - AggregateFunctionCountNotNullUnary(const DataTypePtr & argument) + AggregateFunctionCountNotNullUnary(const DataTypePtr & argument, const Array & params) + : IAggregateFunctionDataHelper({argument}, params) { if (!argument->isNullable()) throw Exception("Logical error: not Nullable data type passed to AggregateFunctionCountNotNullUnary", ErrorCodes::LOGICAL_ERROR); @@ -120,7 +123,8 @@ public: class AggregateFunctionCountNotNullVariadic final : public IAggregateFunctionDataHelper { public: - AggregateFunctionCountNotNullVariadic(const DataTypes & arguments) + AggregateFunctionCountNotNullVariadic(const DataTypes & arguments, const Array & params) + : IAggregateFunctionDataHelper(arguments, params) { number_of_arguments = arguments.size(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionEntropy.cpp b/dbms/src/AggregateFunctions/AggregateFunctionEntropy.cpp index 2f9910c97de..7ea15e11b72 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionEntropy.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionEntropy.cpp @@ -26,12 +26,12 @@ AggregateFunctionPtr createAggregateFunctionEntropy(const std::string & name, co if (num_args == 1) { /// Specialized implementation for single argument of numeric type. - if (auto res = createWithNumericBasedType(*argument_types[0], num_args)) + if (auto res = createWithNumericBasedType(*argument_types[0], argument_types)) return AggregateFunctionPtr(res); } /// Generic implementation for other types or for multiple arguments. - return std::make_shared>(num_args); + return std::make_shared>(argument_types); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h b/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h index 1adeefc6397..91ec6d4d5a6 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionEntropy.h @@ -97,7 +97,9 @@ private: size_t num_args; public: - AggregateFunctionEntropy(size_t num_args) : num_args(num_args) + AggregateFunctionEntropy(const DataTypes & argument_types) + : IAggregateFunctionDataHelper, AggregateFunctionEntropy>(argument_types, {}) + , num_args(argument_types.size()) { } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 932d6615385..6aeaaef2bfa 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -86,17 +86,12 @@ AggregateFunctionPtr AggregateFunctionFactory::get( [](const auto & type) { return type->onlyNull(); })) nested_function = getImpl(name, nested_types, parameters, recursion_level); - auto res = combinator->transformAggregateFunction(nested_function, type_without_low_cardinality, parameters); - res->setArguments(type_without_low_cardinality, parameters); - return res; + return combinator->transformAggregateFunction(nested_function, argument_types, parameters); } auto res = getImpl(name, type_without_low_cardinality, parameters, recursion_level); if (!res) throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR); - - res->setArguments(type_without_low_cardinality, parameters); - return res; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionForEach.h b/dbms/src/AggregateFunctions/AggregateFunctionForEach.h index 519d1911a8a..39a52a7fa6e 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionForEach.h @@ -97,7 +97,8 @@ private: public: AggregateFunctionForEach(AggregateFunctionPtr nested_, const DataTypes & arguments) - : nested_func(nested_), num_arguments(arguments.size()) + : IAggregateFunctionDataHelper(arguments, {}) + , nested_func(nested_), num_arguments(arguments.size()) { nested_size_of_data = nested_func->sizeOfData(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h index 26708c87520..c496e90844d 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -48,12 +48,13 @@ class GroupArrayNumericImpl final : public IAggregateFunctionDataHelper, GroupArrayNumericImpl> { static constexpr bool limit_num_elems = Tlimit_num_elems::value; - DataTypePtr data_type; + DataTypePtr & data_type; UInt64 max_elems; public: explicit GroupArrayNumericImpl(const DataTypePtr & data_type_, UInt64 max_elems_ = std::numeric_limits::max()) - : data_type(data_type_), max_elems(max_elems_) {} + : IAggregateFunctionDataHelper, GroupArrayNumericImpl>({data_type}, {}) + , data_type(argument_types[0]), max_elems(max_elems_) {} String getName() const override { return "groupArray"; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp index bc8fac86d6d..ea42c129dea 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp @@ -13,6 +13,10 @@ namespace AggregateFunctionPtr createAggregateFunctionGroupArrayInsertAt(const std::string & name, const DataTypes & argument_types, const Array & parameters) { assertBinary(name, argument_types); + + if (argument_types.size() != 2) + throw Exception("Aggregate function groupArrayInsertAt requires two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + return std::make_shared(argument_types, parameters); } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h index 90b19266e4c..c7dab21a4cb 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h @@ -54,12 +54,14 @@ class AggregateFunctionGroupArrayInsertAtGeneric final : public IAggregateFunctionDataHelper { private: - DataTypePtr type; + DataTypePtr & type; Field default_value; UInt64 length_to_resize = 0; /// zero means - do not do resizing. public: AggregateFunctionGroupArrayInsertAtGeneric(const DataTypes & arguments, const Array & params) + : IAggregateFunctionDataHelper(arguments, params) + , type(argument_types[0]) { if (!params.empty()) { @@ -76,14 +78,9 @@ public: } } - if (arguments.size() != 2) - throw Exception("Aggregate function " + getName() + " requires two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!isUnsignedInteger(arguments[1])) throw Exception("Second argument of aggregate function " + getName() + " must be integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - type = arguments.front(); - if (default_value.isNull()) default_value = type->getDefault(); else diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp index a84ba2b28a2..f80a45afaa9 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp @@ -15,11 +15,15 @@ namespace /// Substitute return type for Date and DateTime class AggregateFunctionGroupUniqArrayDate : public AggregateFunctionGroupUniqArray { +public: + AggregateFunctionGroupUniqArrayDate(const DataTypePtr & argument_type) : AggregateFunctionGroupUniqArray(argument_type) {} DataTypePtr getReturnType() const override { return std::make_shared(std::make_shared()); } }; class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUniqArray { +public: + AggregateFunctionGroupUniqArrayDateTime(const DataTypePtr & argument_type) : AggregateFunctionGroupUniqArray(argument_type) {} DataTypePtr getReturnType() const override { return std::make_shared(std::make_shared()); } }; @@ -27,8 +31,8 @@ class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUni static IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type) { WhichDataType which(argument_type); - if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate; - else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupUniqArrayDateTime; + if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate(argument_type); + else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupUniqArrayDateTime(argument_type); else { /// Check that we can use plain version of AggreagteFunctionGroupUniqArrayGeneric diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index b638996f553..c0ef1fe0fa8 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -44,6 +44,9 @@ private: using State = AggregateFunctionGroupUniqArrayData; public: + AggregateFunctionGroupUniqArray(const DataTypePtr & argument_type) + : IAggregateFunctionDataHelper, AggregateFunctionGroupUniqArray>({argument_type}, {}) {} + String getName() const override { return "groupUniqArray"; } DataTypePtr getReturnType() const override @@ -115,7 +118,7 @@ template class AggreagteFunctionGroupUniqArrayGeneric : public IAggregateFunctionDataHelper> { - DataTypePtr input_data_type; + DataTypePtr & input_data_type; using State = AggreagteFunctionGroupUniqArrayGenericData; @@ -125,7 +128,8 @@ class AggreagteFunctionGroupUniqArrayGeneric public: AggreagteFunctionGroupUniqArrayGeneric(const DataTypePtr & input_data_type) - : input_data_type(input_data_type) {} + : IAggregateFunctionDataHelper>({input_data_type}, {}) + , input_data_type(argument_types[0]) {} String getName() const override { return "groupUniqArray"; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp index 05c4fe86320..384298b16a8 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.cpp @@ -39,7 +39,7 @@ AggregateFunctionPtr createAggregateFunctionHistogram(const std::string & name, throw Exception("Bin count should be positive", ErrorCodes::BAD_ARGUMENTS); assertUnary(name, arguments); - AggregateFunctionPtr res(createWithNumericType(*arguments[0], bins_count)); + AggregateFunctionPtr res(createWithNumericType(*arguments[0], bins_count, arguments, params)); if (!res) throw Exception("Illegal type " + arguments[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h index 3d03821cc65..60385f4788a 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -304,8 +304,9 @@ private: const UInt32 max_bins; public: - AggregateFunctionHistogram(UInt32 max_bins) - : max_bins(max_bins) + AggregateFunctionHistogram(const DataTypes & arguments, const Array & params, UInt32 max_bins) + : IAggregateFunctionDataHelper>(arguments, params) + , max_bins(max_bins) { } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionIf.h b/dbms/src/AggregateFunctions/AggregateFunctionIf.h index 594193eac87..8daf9505ae6 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionIf.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionIf.h @@ -28,7 +28,8 @@ private: public: AggregateFunctionIf(AggregateFunctionPtr nested, const DataTypes & types) - : nested_func(nested), num_arguments(types.size()) + : IAggregateFunctionHelper(types, nested->getParameters()) + , nested_func(nested), num_arguments(types.size()) { if (num_arguments == 0) throw Exception("Aggregate function " + getName() + " require at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 9b81ce01f30..dbb727b7d9a 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -59,7 +59,7 @@ private: public: AggregateFunctionIntersectionsMax(AggregateFunctionIntersectionsKind kind_, const DataTypes & arguments) - : kind(kind_) + : IAggregateFunctionDataHelper, AggregateFunctionIntersectionsMax>(arguments, {}), kind(kind_) { if (!isNumber(arguments[0])) throw Exception{getName() + ": first argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMerge.cpp b/dbms/src/AggregateFunctions/AggregateFunctionMerge.cpp index 256c7bc9a84..f9c2eb8c9dd 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMerge.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionMerge.cpp @@ -47,7 +47,7 @@ public: + ", because it corresponds to different aggregate function: " + function->getFunctionName() + " instead of " + nested_function->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(nested_function, *argument); + return std::make_shared(nested_function, argument); } }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMerge.h b/dbms/src/AggregateFunctions/AggregateFunctionMerge.h index 2d92db98e17..c94d4d3cf3c 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMerge.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMerge.h @@ -22,13 +22,14 @@ private: AggregateFunctionPtr nested_func; public: - AggregateFunctionMerge(const AggregateFunctionPtr & nested_, const IDataType & argument) - : nested_func(nested_) + AggregateFunctionMerge(const AggregateFunctionPtr & nested_, const DataTypePtr & argument) + : IAggregateFunctionHelper({argument}, nested_->getParameters()) + , nested_func(nested_) { - const DataTypeAggregateFunction * data_type = typeid_cast(&argument); + const DataTypeAggregateFunction * data_type = typeid_cast(argument.get()); if (!data_type || data_type->getFunctionName() != nested_func->getName()) - throw Exception("Illegal type " + argument.getName() + " of argument for aggregate function " + getName(), + throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 51d1e8d1dd7..426ee8ee479 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -676,10 +676,12 @@ template class AggregateFunctionsSingleValue final : public IAggregateFunctionDataHelper> { private: - DataTypePtr type; + DataTypePtr & type; public: - AggregateFunctionsSingleValue(const DataTypePtr & type) : type(type) + AggregateFunctionsSingleValue(const DataTypePtr & type) + : IAggregateFunctionDataHelper>({type}, {}) + , type(argument_types[0]) { if (StringRef(Data::name()) == StringRef("min") || StringRef(Data::name()) == StringRef("max")) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionNothing.h b/dbms/src/AggregateFunctions/AggregateFunctionNothing.h index 3a98807bb4a..aa54d95f158 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionNothing.h @@ -15,6 +15,9 @@ namespace DB class AggregateFunctionNothing final : public IAggregateFunctionHelper { public: + AggregateFunctionNothing(const DataTypes & arguments, const Array & params) + : IAggregateFunctionHelper(arguments, params) {} + String getName() const override { return "nothing"; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionNull.cpp b/dbms/src/AggregateFunctions/AggregateFunctionNull.cpp index 6ce7d94d970..7011ebbde09 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionNull.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionNull.cpp @@ -30,7 +30,7 @@ public: } AggregateFunctionPtr transformAggregateFunction( - const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array &) const override + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params) const override { bool has_nullable_types = false; bool has_null_types = false; @@ -55,29 +55,29 @@ public: if (nested_function && nested_function->getName() == "count") { if (arguments.size() == 1) - return std::make_shared(arguments[0]); + return std::make_shared(arguments[0], params); else - return std::make_shared(arguments); + return std::make_shared(arguments, params); } if (has_null_types) - return std::make_shared(); + return std::make_shared(arguments, params); bool return_type_is_nullable = nested_function->getReturnType()->canBeInsideNullable(); if (arguments.size() == 1) { if (return_type_is_nullable) - return std::make_shared>(nested_function); + return std::make_shared>(nested_function, arguments, params); else - return std::make_shared>(nested_function); + return std::make_shared>(nested_function, arguments, params); } else { if (return_type_is_nullable) - return std::make_shared>(nested_function, arguments); + return std::make_shared>(nested_function, arguments, params); else - return std::make_shared>(nested_function, arguments); + return std::make_shared>(nested_function, arguments, params); } } }; diff --git a/dbms/src/AggregateFunctions/AggregateFunctionNull.h b/dbms/src/AggregateFunctions/AggregateFunctionNull.h index c8676230500..ab4b5b27844 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionNull.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionNull.h @@ -68,8 +68,8 @@ protected: } public: - AggregateFunctionNullBase(AggregateFunctionPtr nested_function_) - : nested_function{nested_function_} + AggregateFunctionNullBase(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : IAggregateFunctionHelper(arguments, params), nested_function{nested_function_} { if (result_is_nullable) prefix_size = nested_function->alignOfData(); @@ -187,8 +187,8 @@ template class AggregateFunctionNullUnary final : public AggregateFunctionNullBase> { public: - AggregateFunctionNullUnary(AggregateFunctionPtr nested_function_) - : AggregateFunctionNullBase>(std::move(nested_function_)) + AggregateFunctionNullUnary(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params) { } @@ -209,8 +209,8 @@ template class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase> { public: - AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments) - : AggregateFunctionNullBase>(std::move(nested_function_)), + AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) + : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size()) { if (number_of_arguments == 1) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h index cee2b6fe0c0..a87f520d395 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -73,11 +73,12 @@ private: /// Used when there are single level to get. Float64 level = 0.5; - DataTypePtr argument_type; + DataTypePtr & argument_type; public: AggregateFunctionQuantile(const DataTypePtr & argument_type, const Array & params) - : levels(params, returns_many), level(levels.levels[0]), argument_type(argument_type) + : IAggregateFunctionDataHelper>({argument_type}, params) + , levels(params, returns_many), level(levels.levels[0]), argument_type(argument_types[0]) { if (!returns_many && levels.size() > 1) throw Exception("Aggregate function " + getName() + " require one parameter or less", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionRetention.h b/dbms/src/AggregateFunctions/AggregateFunctionRetention.h index 688f7f1404c..525a4d848d2 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionRetention.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionRetention.h @@ -76,6 +76,7 @@ public: } AggregateFunctionRetention(const DataTypes & arguments) + : IAggregateFunctionDataHelper(arguments, {}) { for (const auto i : ext::range(0, arguments.size())) { diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.cpp index 0b7a4b6b357..be139d9e633 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.cpp @@ -19,7 +19,7 @@ AggregateFunctionPtr createAggregateFunctionSequenceCount(const std::string & na ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; String pattern = params.front().safeGet(); - return std::make_shared(argument_types, pattern); + return std::make_shared(argument_types, params, pattern); } AggregateFunctionPtr createAggregateFunctionSequenceMatch(const std::string & name, const DataTypes & argument_types, const Array & params) @@ -29,7 +29,7 @@ AggregateFunctionPtr createAggregateFunctionSequenceMatch(const std::string & na ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; String pattern = params.front().safeGet(); - return std::make_shared(argument_types, pattern); + return std::make_shared(argument_types, params, pattern); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h index 86627a453c2..5c443c72b63 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -139,8 +139,9 @@ template class AggregateFunctionSequenceBase : public IAggregateFunctionDataHelper { public: - AggregateFunctionSequenceBase(const DataTypes & arguments, const String & pattern) - : pattern(pattern) + AggregateFunctionSequenceBase(const DataTypes & arguments, const Array & params, const String & pattern) + : IAggregateFunctionDataHelper(arguments, params) + , pattern(pattern) { arg_count = arguments.size(); @@ -578,6 +579,9 @@ private: class AggregateFunctionSequenceMatch final : public AggregateFunctionSequenceBase { public: + AggregateFunctionSequenceMatch(const DataTypes & arguments, const Array & params, const String & pattern) + : AggregateFunctionSequenceBase(arguments, params, pattern) {} + using AggregateFunctionSequenceBase::AggregateFunctionSequenceBase; String getName() const override { return "sequenceMatch"; } @@ -603,6 +607,9 @@ public: class AggregateFunctionSequenceCount final : public AggregateFunctionSequenceBase { public: + AggregateFunctionSequenceCount(const DataTypes & arguments, const Array & params, const String & pattern) + : AggregateFunctionSequenceBase(arguments, params, pattern) {} + using AggregateFunctionSequenceBase::AggregateFunctionSequenceBase; String getName() const override { return "sequenceCount"; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionState.h b/dbms/src/AggregateFunctions/AggregateFunctionState.h index 30755ce3896..2d8e5c6a537 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionState.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionState.h @@ -24,7 +24,8 @@ private: public: AggregateFunctionState(AggregateFunctionPtr nested, const DataTypes & arguments, const Array & params) - : nested_func(nested), arguments(arguments), params(params) {} + : IAggregateFunctionHelper(arguments, params) + , nested_func(nested), arguments(arguments), params(params) {} String getName() const override { diff --git a/dbms/src/AggregateFunctions/AggregateFunctionStatistics.cpp b/dbms/src/AggregateFunctions/AggregateFunctionStatistics.cpp index ae73013d29d..1530ad25cf3 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionStatistics.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionStatistics.cpp @@ -21,7 +21,7 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & assertNoParameters(name, parameters); assertUnary(name, argument_types); - AggregateFunctionPtr res(createWithNumericType(*argument_types[0])); + AggregateFunctionPtr res(createWithNumericType(*argument_types[0], argument_types[0])); if (!res) throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -35,7 +35,7 @@ AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & assertNoParameters(name, parameters); assertBinary(name, argument_types); - AggregateFunctionPtr res(createWithTwoNumericTypes(*argument_types[0], *argument_types[1])); + AggregateFunctionPtr res(createWithTwoNumericTypes(*argument_types[0], *argument_types[1], argument_types)); if (!res) throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName() + " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h b/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h index 82d34fc2954..d1112ec0831 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionStatistics.h @@ -111,6 +111,9 @@ class AggregateFunctionVariance final : public IAggregateFunctionDataHelper, AggregateFunctionVariance> { public: + AggregateFunctionVariance(const DataTypePtr & arg) + : IAggregateFunctionDataHelper, AggregateFunctionVariance>({arg}, {}) {} + String getName() const override { return Op::name; } DataTypePtr getReturnType() const override @@ -361,6 +364,10 @@ class AggregateFunctionCovariance final AggregateFunctionCovariance> { public: + AggregateFunctionCovariance(const DataTypes & args) : IAggregateFunctionDataHelper< + CovarianceData, + AggregateFunctionCovariance>(args, {}) {} + String getName() const override { return Op::name; } DataTypePtr getReturnType() const override diff --git a/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h b/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h index 0580a5131a2..4ab6a4d51ed 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h @@ -288,12 +288,14 @@ public: using ResultType = typename StatFunc::ResultType; using ColVecResult = ColumnVector; - AggregateFunctionVarianceSimple() - : src_scale(0) + AggregateFunctionVarianceSimple(const DataTypes & argument_types) + : IAggregateFunctionDataHelper>(argument_types, {}) + , src_scale(0) {} - AggregateFunctionVarianceSimple(const IDataType & data_type) - : src_scale(getDecimalScale(data_type)) + AggregateFunctionVarianceSimple(const IDataType & data_type, const DataTypes & argument_types) + : IAggregateFunctionDataHelper>(argument_types, {}) + , src_scale(getDecimalScale(data_type)) {} String getName() const override diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSum.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSum.cpp index f21c60eeae6..5e060d7b7df 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSum.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSum.cpp @@ -50,9 +50,9 @@ AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const AggregateFunctionPtr res; DataTypePtr data_type = argument_types[0]; if (isDecimal(data_type)) - res.reset(createWithDecimalType(*data_type, *data_type)); + res.reset(createWithDecimalType(*data_type, *data_type, argument_types)); else - res.reset(createWithNumericType(*data_type)); + res.reset(createWithNumericType(*data_type, argument_types)); if (!res) throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSum.h b/dbms/src/AggregateFunctions/AggregateFunctionSum.h index 5bd2d10917a..1860088cd93 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSum.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSum.h @@ -102,12 +102,14 @@ public: String getName() const override { return "sum"; } - AggregateFunctionSum() - : scale(0) + AggregateFunctionSum(const DataTypes & argument_types) + : IAggregateFunctionDataHelper>(argument_types, {}) + , scale(0) {} - AggregateFunctionSum(const IDataType & data_type) - : scale(getDecimalScale(data_type)) + AggregateFunctionSum(const IDataType & data_type, const DataTypes & argument_types) + : IAggregateFunctionDataHelper>(argument_types, {}) + , scale(getDecimalScale(data_type)) {} DataTypePtr getReturnType() const override diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 75cd62c00f1..5a10ae62324 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -80,7 +80,7 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con auto [keys_type, values_types] = parseArguments(name, arguments); - AggregateFunctionPtr res(createWithNumericBasedType(*keys_type, keys_type, values_types)); + AggregateFunctionPtr res(createWithNumericBasedType(*keys_type, keys_type, values_types, arguments)); if (!res) res.reset(createWithDecimalType(*keys_type, keys_type, values_types)); if (!res) @@ -103,7 +103,7 @@ AggregateFunctionPtr createAggregateFunctionSumMapFiltered(const std::string & n auto [keys_type, values_types] = parseArguments(name, arguments); - AggregateFunctionPtr res(createWithNumericBasedType(*keys_type, keys_type, values_types, keys_to_keep)); + AggregateFunctionPtr res(createWithNumericBasedType(*keys_type, keys_type, values_types, keys_to_keep, arguments, params)); if (!res) res.reset(createWithDecimalType(*keys_type, keys_type, values_types, keys_to_keep)); if (!res) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h index c239b74630e..ef6cae9babc 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -61,8 +61,11 @@ private: DataTypes values_types; public: - AggregateFunctionSumMapBase(const DataTypePtr & keys_type, const DataTypes & values_types) - : keys_type(keys_type), values_types(values_types) {} + AggregateFunctionSumMapBase( + const DataTypePtr & keys_type, const DataTypes & values_types, + const DataTypes & argument_types, const Array & params) + : IAggregateFunctionDataHelper>, Derived>(argument_types, params) + , keys_type(keys_type), values_types(values_types) {} String getName() const override { return "sumMap"; } @@ -271,8 +274,8 @@ private: using Base = AggregateFunctionSumMapBase; public: - AggregateFunctionSumMap(const DataTypePtr & keys_type, DataTypes & values_types) - : Base{keys_type, values_types} + AggregateFunctionSumMap(const DataTypePtr & keys_type, DataTypes & values_types, const DataTypes & argument_types) + : Base{keys_type, values_types, argument_types, {}} {} String getName() const override { return "sumMap"; } @@ -291,8 +294,10 @@ private: std::unordered_set keys_to_keep; public: - AggregateFunctionSumMapFiltered(const DataTypePtr & keys_type, const DataTypes & values_types, const Array & keys_to_keep_) - : Base{keys_type, values_types} + AggregateFunctionSumMapFiltered( + const DataTypePtr & keys_type, const DataTypes & values_types, const Array & keys_to_keep_, + const DataTypes & argument_types, const Array & params) + : Base{keys_type, values_types, argument_types, params} { keys_to_keep.reserve(keys_to_keep_.size()); for (const Field & f : keys_to_keep_) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionTopK.cpp b/dbms/src/AggregateFunctions/AggregateFunctionTopK.cpp index 168dba4ebd5..04e74c17434 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -39,19 +39,19 @@ class AggregateFunctionTopKDateTime : public AggregateFunctionTopK -static IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, UInt64 threshold) +static IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, UInt64 threshold, const Array & params) { WhichDataType which(argument_type); if (which.idx == TypeIndex::Date) - return new AggregateFunctionTopKDate(threshold); + return new AggregateFunctionTopKDate(threshold, {argument_type}, params); if (which.idx == TypeIndex::DateTime) - return new AggregateFunctionTopKDateTime(threshold); + return new AggregateFunctionTopKDateTime(threshold, {argument_type}, params); /// Check that we can use plain version of AggregateFunctionTopKGeneric if (argument_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) - return new AggregateFunctionTopKGeneric(threshold, argument_type); + return new AggregateFunctionTopKGeneric(threshold, argument_type, params); else - return new AggregateFunctionTopKGeneric(threshold, argument_type); + return new AggregateFunctionTopKGeneric(threshold, argument_type, params); } @@ -90,10 +90,10 @@ AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const threshold = k; } - AggregateFunctionPtr res(createWithNumericType(*argument_types[0], threshold)); + AggregateFunctionPtr res(createWithNumericType(*argument_types[0], threshold, argument_types, params)); if (!res) - res = AggregateFunctionPtr(createWithExtraTypes(argument_types[0], threshold)); + res = AggregateFunctionPtr(createWithExtraTypes(argument_types[0], threshold, params)); if (!res) throw Exception("Illegal type " + argument_types[0]->getName() + diff --git a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h index 09897f5ccd2..846a3e2b2a1 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h @@ -48,8 +48,9 @@ protected: UInt64 reserved; public: - AggregateFunctionTopK(UInt64 threshold) - : threshold(threshold), reserved(TOP_K_LOAD_FACTOR * threshold) {} + AggregateFunctionTopK(UInt64 threshold, const DataTypes & argument_types, const Array & params) + : IAggregateFunctionDataHelper, AggregateFunctionTopK>(argument_types, params) + , threshold(threshold), reserved(TOP_K_LOAD_FACTOR * threshold) {} String getName() const override { return is_weighted ? "topKWeighted" : "topK"; } @@ -136,13 +137,15 @@ private: UInt64 threshold; UInt64 reserved; - DataTypePtr input_data_type; + DataTypePtr & input_data_type; static void deserializeAndInsert(StringRef str, IColumn & data_to); public: - AggregateFunctionTopKGeneric(UInt64 threshold, const DataTypePtr & input_data_type) - : threshold(threshold), reserved(TOP_K_LOAD_FACTOR * threshold), input_data_type(input_data_type) {} + AggregateFunctionTopKGeneric( + UInt64 threshold, const DataTypePtr & input_data_type, const Array & params) + : IAggregateFunctionDataHelper>({input_data_type}, params) + , threshold(threshold), reserved(TOP_K_LOAD_FACTOR * threshold), input_data_type(argument_types[0]) {} String getName() const override { return is_weighted ? "topKWeighted" : "topK"; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp index 6b63a719b8f..eaf021d8735 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -43,19 +43,19 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const { const IDataType & argument_type = *argument_types[0]; - AggregateFunctionPtr res(createWithNumericType(*argument_types[0])); + AggregateFunctionPtr res(createWithNumericType(*argument_types[0], argument_types)); WhichDataType which(argument_type); if (res) return res; else if (which.isDate()) - return std::make_shared>(); + return std::make_shared>(argument_types); else if (which.isDateTime()) - return std::make_shared>(); + return std::make_shared>(argument_types); else if (which.isStringOrFixedString()) - return std::make_shared>(); + return std::make_shared>(argument_types); else if (which.isUUID()) - return std::make_shared>(); + return std::make_shared>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) @@ -89,19 +89,19 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const { const IDataType & argument_type = *argument_types[0]; - AggregateFunctionPtr res(createWithNumericType(*argument_types[0])); + AggregateFunctionPtr res(createWithNumericType(*argument_types[0], argument_types)); WhichDataType which(argument_type); if (res) return res; else if (which.isDate()) - return std::make_shared>>(); + return std::make_shared>>(argument_types); else if (which.isDateTime()) - return std::make_shared>>(); + return std::make_shared>>(argument_types); else if (which.isStringOrFixedString()) return std::make_shared>>(); else if (which.isUUID()) - return std::make_shared>>(); + return std::make_shared>>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h index fea79a920a9..56a855aabb9 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h @@ -209,6 +209,9 @@ template class AggregateFunctionUniq final : public IAggregateFunctionDataHelper> { public: + AggregateFunctionUniq(const DataTypes & argument_types) + : IAggregateFunctionDataHelper>(argument_types, {}) {} + String getName() const override { return Data::getName(); } DataTypePtr getReturnType() const override @@ -257,6 +260,7 @@ private: public: AggregateFunctionUniqVariadic(const DataTypes & arguments) + : IAggregateFunctionDataHelper>(arguments) { if (argument_is_tuple) num_args = typeid_cast(*arguments[0]).getElements().size(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index 90b84d3b927..38982b8130e 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -28,7 +28,7 @@ namespace }; template - AggregateFunctionPtr createAggregateFunctionWithK(const DataTypes & argument_types) + AggregateFunctionPtr createAggregateFunctionWithK(const DataTypes & argument_types, const Array & params) { /// We use exact hash function if the arguments are not contiguous in memory, because only exact hash function has support for this case. bool use_exact_hash_function = !isAllArgumentsContiguousInMemory(argument_types); @@ -37,33 +37,33 @@ namespace { const IDataType & argument_type = *argument_types[0]; - AggregateFunctionPtr res(createWithNumericType::template AggregateFunction>(*argument_types[0])); + AggregateFunctionPtr res(createWithNumericType::template AggregateFunction>(*argument_types[0], argument_types, params)); WhichDataType which(argument_type); if (res) return res; else if (which.isDate()) - return std::make_shared::template AggregateFunction>(); + return std::make_shared::template AggregateFunction>(argument_types, params); else if (which.isDateTime()) - return std::make_shared::template AggregateFunction>(); + return std::make_shared::template AggregateFunction>(argument_types, params); else if (which.isStringOrFixedString()) - return std::make_shared::template AggregateFunction>(); + return std::make_shared::template AggregateFunction>(argument_types, params); else if (which.isUUID()) - return std::make_shared::template AggregateFunction>(); + return std::make_shared::template AggregateFunction>(argument_types, params); else if (which.isTuple()) { if (use_exact_hash_function) - return std::make_shared::template AggregateFunctionVariadic>(argument_types); + return std::make_shared::template AggregateFunctionVariadic>(argument_types, params); else - return std::make_shared::template AggregateFunctionVariadic>(argument_types); + return std::make_shared::template AggregateFunctionVariadic>(argument_types, params); } } /// "Variadic" method also works as a fallback generic case for a single argument. if (use_exact_hash_function) - return std::make_shared::template AggregateFunctionVariadic>(argument_types); + return std::make_shared::template AggregateFunctionVariadic>(argument_types, params); else - return std::make_shared::template AggregateFunctionVariadic>(argument_types); + return std::make_shared::template AggregateFunctionVariadic>(argument_types, params); } AggregateFunctionPtr createAggregateFunctionUniqCombined( @@ -95,23 +95,23 @@ namespace switch (precision) { case 12: - return createAggregateFunctionWithK<12>(argument_types); + return createAggregateFunctionWithK<12>(argument_types, params); case 13: - return createAggregateFunctionWithK<13>(argument_types); + return createAggregateFunctionWithK<13>(argument_types, params); case 14: - return createAggregateFunctionWithK<14>(argument_types); + return createAggregateFunctionWithK<14>(argument_types, params); case 15: - return createAggregateFunctionWithK<15>(argument_types); + return createAggregateFunctionWithK<15>(argument_types, params); case 16: - return createAggregateFunctionWithK<16>(argument_types); + return createAggregateFunctionWithK<16>(argument_types, params); case 17: - return createAggregateFunctionWithK<17>(argument_types); + return createAggregateFunctionWithK<17>(argument_types, params); case 18: - return createAggregateFunctionWithK<18>(argument_types); + return createAggregateFunctionWithK<18>(argument_types, params); case 19: - return createAggregateFunctionWithK<19>(argument_types); + return createAggregateFunctionWithK<19>(argument_types, params); case 20: - return createAggregateFunctionWithK<20>(argument_types); + return createAggregateFunctionWithK<20>(argument_types, params); } __builtin_unreachable(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h index 001f4e7f289..3b7aee95186 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -114,6 +114,9 @@ class AggregateFunctionUniqCombined final : public IAggregateFunctionDataHelper, AggregateFunctionUniqCombined> { public: + AggregateFunctionUniqCombined(const DataTypes & argument_types, const Array & params) + : IAggregateFunctionDataHelper, AggregateFunctionUniqCombined>(argument_types, params) {} + String getName() const override { return "uniqCombined"; @@ -176,7 +179,9 @@ private: size_t num_args = 0; public: - explicit AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments) + explicit AggregateFunctionUniqCombinedVariadic(const DataTypes & arguments, const Array & params) + : IAggregateFunctionDataHelper, + AggregateFunctionUniqCombinedVariadic>(arguments, params) { if (argument_is_tuple) num_args = typeid_cast(*arguments[0]).getElements().size(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp index b9cdcaa4eae..ba4f337839e 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.cpp @@ -52,33 +52,33 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c { const IDataType & argument_type = *argument_types[0]; - AggregateFunctionPtr res(createWithNumericType(*argument_types[0], threshold)); + AggregateFunctionPtr res(createWithNumericType(*argument_types[0], threshold, argument_types, params)); WhichDataType which(argument_type); if (res) return res; else if (which.isDate()) - return std::make_shared>(threshold); + return std::make_shared>(threshold, argument_types, params); else if (which.isDateTime()) - return std::make_shared>(threshold); + return std::make_shared>(threshold, argument_types, params); else if (which.isStringOrFixedString()) - return std::make_shared>(threshold); + return std::make_shared>(threshold, argument_types, params); else if (which.isUUID()) - return std::make_shared>(threshold); + return std::make_shared>(threshold, argument_types, params); else if (which.isTuple()) { if (use_exact_hash_function) - return std::make_shared>(argument_types, threshold); + return std::make_shared>(argument_types, params, threshold); else - return std::make_shared>(argument_types, threshold); + return std::make_shared>(argument_types, params, threshold); } } /// "Variadic" method also works as a fallback generic case for single argument. if (use_exact_hash_function) - return std::make_shared>(argument_types, threshold); + return std::make_shared>(argument_types, params, threshold); else - return std::make_shared>(argument_types, threshold); + return std::make_shared>(argument_types, params, threshold); } } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h b/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h index 6b6a645024a..477a729894d 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniqUpTo.h @@ -136,8 +136,9 @@ private: UInt8 threshold; public: - AggregateFunctionUniqUpTo(UInt8 threshold) - : threshold(threshold) + AggregateFunctionUniqUpTo(UInt8 threshold, const DataTypes & argument_types, const Array & params) + : IAggregateFunctionDataHelper, AggregateFunctionUniqUpTo>(argument_types, params) + , threshold(threshold) { } @@ -195,8 +196,9 @@ private: UInt8 threshold; public: - AggregateFunctionUniqUpToVariadic(const DataTypes & arguments, UInt8 threshold) - : threshold(threshold) + AggregateFunctionUniqUpToVariadic(const DataTypes & arguments, const Array & params, UInt8 threshold) + : IAggregateFunctionDataHelper, AggregateFunctionUniqUpToVariadic>(arguments, params) + , threshold(threshold) { if (argument_is_tuple) num_args = typeid_cast(*arguments[0]).getElements().size(); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 317637b1b69..556f9bb1ae1 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -189,6 +189,7 @@ public: } AggregateFunctionWindowFunnel(const DataTypes & arguments, const Array & params) + : IAggregateFunctionDataHelper(arguments, params) { const auto time_arg = arguments.front().get(); if (!WhichDataType(time_arg).isDateTime() && !WhichDataType(time_arg).isUInt32()) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionsStatisticsSimple.cpp b/dbms/src/AggregateFunctions/AggregateFunctionsStatisticsSimple.cpp index 4159403afc7..1fafa6e00c9 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionsStatisticsSimple.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionsStatisticsSimple.cpp @@ -24,9 +24,9 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string & AggregateFunctionPtr res; DataTypePtr data_type = argument_types[0]; if (isDecimal(data_type)) - res.reset(createWithDecimalType(*data_type, *data_type)); + res.reset(createWithDecimalType(*data_type, *data_type, argument_types)); else - res.reset(createWithNumericType(*data_type)); + res.reset(createWithNumericType(*data_type, argument_types)); if (!res) throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, @@ -40,7 +40,7 @@ AggregateFunctionPtr createAggregateFunctionStatisticsBinary(const std::string & assertNoParameters(name, parameters); assertBinary(name, argument_types); - AggregateFunctionPtr res(createWithTwoNumericTypes(*argument_types[0], *argument_types[1])); + AggregateFunctionPtr res(createWithTwoNumericTypes(*argument_types[0], *argument_types[1], argument_types)); if (!res) throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName() + " of arguments for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/AggregateFunctions/IAggregateFunction.h b/dbms/src/AggregateFunctions/IAggregateFunction.h index f5def066058..17620f7493d 100644 --- a/dbms/src/AggregateFunctions/IAggregateFunction.h +++ b/dbms/src/AggregateFunctions/IAggregateFunction.h @@ -37,6 +37,9 @@ using ConstAggregateDataPtr = const char *; class IAggregateFunction { public: + IAggregateFunction(const DataTypes & argument_types_, const Array & parameters_) + : argument_types(argument_types_), parameters(parameters_) {} + /// Get main function name. virtual String getName() const = 0; @@ -112,17 +115,9 @@ public: const DataTypes & getArgumentTypes() const { return argument_types; } const Array & getParameters() const { return parameters; } -private: +protected: DataTypes argument_types; Array parameters; - - friend class AggregateFunctionFactory; - - void setArguments(DataTypes argument_types_, Array parameters_) - { - argument_types = std::move(argument_types_); - parameters = std::move(parameters_); - } }; @@ -137,6 +132,8 @@ private: } public: + IAggregateFunctionHelper(const DataTypes & argument_types_, const Array & parameters_) + : IAggregateFunction(argument_types_, parameters_) {} AddFunc getAddressOfAddFunction() const override { return &addFree; } }; @@ -152,6 +149,10 @@ protected: static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } public: + + IAggregateFunctionDataHelper(const DataTypes & argument_types_, const Array & parameters_) + : IAggregateFunctionHelper(argument_types_, parameters_) {} + void create(AggregateDataPtr place) const override { new (place) Data; From bbfd6c502c037b09902441a9631f29a736f38c10 Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 11 Feb 2019 22:33:04 +0300 Subject: [PATCH 09/40] test for issue-3998 --- dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference | 2 ++ dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql | 3 +++ 2 files changed, 5 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference index 204377075a9..c79c5b1fbc7 100644 --- a/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference +++ b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference @@ -3,3 +3,5 @@ 11 11 11 12 12 11 +0 +1 diff --git a/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql index b1cfa3780a2..af62eef03ca 100644 --- a/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql +++ b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql @@ -3,3 +3,6 @@ select s.a + 1 as a, s.a as b from (select 10 as a) s; select s.a + 1 as a, s.a + 1 as b from (select 10 as a) s; select s.a + 1 as b, s.a + 2 as a from (select 10 as a) s; select s.a + 2 as b, s.a + 1 as a from (select 10 as a) s; + +SELECT 0 as t FROM (SELECT 1 as t) as inn WHERE inn.t = 1; +SELECT sum(value) as value FROM (SELECT 1 as value) as data WHERE data.value > 0; From a493f9ee7d2df969ad8f4fc0d78c0046548225eb Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 11 Feb 2019 22:39:11 +0300 Subject: [PATCH 10/40] add test for issue-3290 --- .../queries/0_stateless/00818_alias_bug_4110.reference | 1 + dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql | 7 +++++++ 2 files changed, 8 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference index c79c5b1fbc7..5186cb8eeff 100644 --- a/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference +++ b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.reference @@ -5,3 +5,4 @@ 12 11 0 1 +123 456 diff --git a/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql index af62eef03ca..7480f137a65 100644 --- a/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql +++ b/dbms/tests/queries/0_stateless/00818_alias_bug_4110.sql @@ -6,3 +6,10 @@ select s.a + 2 as b, s.a + 1 as a from (select 10 as a) s; SELECT 0 as t FROM (SELECT 1 as t) as inn WHERE inn.t = 1; SELECT sum(value) as value FROM (SELECT 1 as value) as data WHERE data.value > 0; + +USE test; +DROP TABLE IF EXISTS test; +CREATE TABLE test (field String, not_field String) ENGINE = Memory; +INSERT INTO test (field, not_field) VALUES ('123', '456') +SELECT test.field AS other_field, test.not_field AS field FROM test; +DROP TABLE test; From 428f7ce27ba808998599b7b3b3b506552e13cd9c Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 12 Feb 2019 00:43:29 +0300 Subject: [PATCH 11/40] Fix split link of dbms/programs/odbc-bridge --- dbms/programs/odbc-bridge/CMakeLists.txt | 13 +++---------- dbms/programs/odbc-bridge/MainHandler.cpp | 2 +- .../odbc-bridge}/ODBCBlockInputStream.cpp | 0 .../odbc-bridge}/ODBCBlockInputStream.h | 2 +- 4 files changed, 5 insertions(+), 12 deletions(-) rename dbms/{src/Dictionaries => programs/odbc-bridge}/ODBCBlockInputStream.cpp (100%) rename dbms/{src/Dictionaries => programs/odbc-bridge}/ODBCBlockInputStream.h (94%) diff --git a/dbms/programs/odbc-bridge/CMakeLists.txt b/dbms/programs/odbc-bridge/CMakeLists.txt index 12062b5a939..3b06e0bc395 100644 --- a/dbms/programs/odbc-bridge/CMakeLists.txt +++ b/dbms/programs/odbc-bridge/CMakeLists.txt @@ -1,13 +1,6 @@ -add_library (clickhouse-odbc-bridge-lib ${LINK_MODE} - PingHandler.cpp - MainHandler.cpp - ColumnInfoHandler.cpp - IdentifierQuoteHandler.cpp - HandlerFactory.cpp - ODBCBridge.cpp - getIdentifierQuote.cpp - validateODBCConnectionString.cpp -) +add_headers_and_sources(clickhouse_odbc_bridge .) + +add_library (clickhouse-odbc-bridge-lib ${LINK_MODE} ${clickhouse_odbc_bridge_sources}) target_link_libraries (clickhouse-odbc-bridge-lib PRIVATE daemon dbms clickhouse_common_io) target_include_directories (clickhouse-odbc-bridge-lib PUBLIC ${ClickHouse_SOURCE_DIR}/libs/libdaemon/include) diff --git a/dbms/programs/odbc-bridge/MainHandler.cpp b/dbms/programs/odbc-bridge/MainHandler.cpp index 8ffedc5a25a..2aebdda3b03 100644 --- a/dbms/programs/odbc-bridge/MainHandler.cpp +++ b/dbms/programs/odbc-bridge/MainHandler.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include "ODBCBlockInputStream.h" #include #include #include diff --git a/dbms/src/Dictionaries/ODBCBlockInputStream.cpp b/dbms/programs/odbc-bridge/ODBCBlockInputStream.cpp similarity index 100% rename from dbms/src/Dictionaries/ODBCBlockInputStream.cpp rename to dbms/programs/odbc-bridge/ODBCBlockInputStream.cpp diff --git a/dbms/src/Dictionaries/ODBCBlockInputStream.h b/dbms/programs/odbc-bridge/ODBCBlockInputStream.h similarity index 94% rename from dbms/src/Dictionaries/ODBCBlockInputStream.h rename to dbms/programs/odbc-bridge/ODBCBlockInputStream.h index 46bfed01a51..e22c245fa47 100644 --- a/dbms/src/Dictionaries/ODBCBlockInputStream.h +++ b/dbms/programs/odbc-bridge/ODBCBlockInputStream.h @@ -6,7 +6,7 @@ #include #include #include -#include "ExternalResultDescription.h" +#include namespace DB From 84a71a2287ceaa394d1870951f620f681ae64372 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Feb 2019 11:42:56 +0300 Subject: [PATCH 12/40] Fix perf test to correct tables --- dbms/tests/performance/columns_hashing/columns_hashing.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/tests/performance/columns_hashing/columns_hashing.xml b/dbms/tests/performance/columns_hashing/columns_hashing.xml index d8cebcb4447..b19fdd9e770 100644 --- a/dbms/tests/performance/columns_hashing/columns_hashing.xml +++ b/dbms/tests/performance/columns_hashing/columns_hashing.xml @@ -38,9 +38,9 @@ --> - - - + + + From 067ee988520c96c11b8026eb09b382afbe7eb461 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Feb 2019 12:13:31 +0300 Subject: [PATCH 13/40] Forward settings in perf test and fix exception in uniq.xml --- dbms/programs/performance-test/PerformanceTest.cpp | 4 ++-- dbms/programs/performance-test/executeQuery.cpp | 4 ++-- dbms/programs/performance-test/executeQuery.h | 4 +++- dbms/tests/performance/date_time/date_time.xml | 1 - dbms/tests/performance/uniq/uniq.xml | 6 +++--- 5 files changed, 10 insertions(+), 9 deletions(-) diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index bf0cf0e8d0f..5d7a342799f 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -240,7 +240,7 @@ void PerformanceTest::runQueries( statistics.startWatches(); try { - executeQuery(connection, query, statistics, stop_conditions, interrupt_listener, context); + executeQuery(connection, query, statistics, stop_conditions, interrupt_listener, context, test_info.settings); if (test_info.exec_type == ExecutionType::Loop) { @@ -254,7 +254,7 @@ void PerformanceTest::runQueries( break; } - executeQuery(connection, query, statistics, stop_conditions, interrupt_listener, context); + executeQuery(connection, query, statistics, stop_conditions, interrupt_listener, context, test_info.settings); } } } diff --git a/dbms/programs/performance-test/executeQuery.cpp b/dbms/programs/performance-test/executeQuery.cpp index 98a1c7a9ef7..58f56d0ba10 100644 --- a/dbms/programs/performance-test/executeQuery.cpp +++ b/dbms/programs/performance-test/executeQuery.cpp @@ -44,14 +44,14 @@ void executeQuery( TestStats & statistics, TestStopConditions & stop_conditions, InterruptListener & interrupt_listener, - Context & context) + Context & context, + const Settings & settings) { statistics.watch_per_query.restart(); statistics.last_query_was_cancelled = false; statistics.last_query_rows_read = 0; statistics.last_query_bytes_read = 0; - Settings settings; RemoteBlockInputStream stream(connection, query, {}, context, &settings); stream.setProgressCallback( diff --git a/dbms/programs/performance-test/executeQuery.h b/dbms/programs/performance-test/executeQuery.h index b1942437e0a..61494ca53b4 100644 --- a/dbms/programs/performance-test/executeQuery.h +++ b/dbms/programs/performance-test/executeQuery.h @@ -4,6 +4,7 @@ #include "TestStopConditions.h" #include #include +#include #include namespace DB @@ -14,5 +15,6 @@ void executeQuery( TestStats & statistics, TestStopConditions & stop_conditions, InterruptListener & interrupt_listener, - Context & context); + Context & context, + const Settings & settings); } diff --git a/dbms/tests/performance/date_time/date_time.xml b/dbms/tests/performance/date_time/date_time.xml index b783e5be808..a31217792dc 100644 --- a/dbms/tests/performance/date_time/date_time.xml +++ b/dbms/tests/performance/date_time/date_time.xml @@ -76,7 +76,6 @@ toISOWeek toISOYear - toStartOfDay toDate toMonday toStartOfMonth diff --git a/dbms/tests/performance/uniq/uniq.xml b/dbms/tests/performance/uniq/uniq.xml index ba9e347ac52..9cb031b42b8 100644 --- a/dbms/tests/performance/uniq/uniq.xml +++ b/dbms/tests/performance/uniq/uniq.xml @@ -9,11 +9,11 @@ - 10000 + 8000 5000 - 20000 + 15000 @@ -22,7 +22,7 @@ - 20000000000 + 30000000000 From ec6a5590fdd99ea24fc42ba41208864bab9ffa1e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Feb 2019 12:31:20 +0300 Subject: [PATCH 14/40] Fix build. --- dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h | 2 +- dbms/src/AggregateFunctions/AggregateFunctionAvg.cpp | 4 ++-- dbms/src/AggregateFunctions/AggregateFunctionAvg.h | 9 +++++---- .../src/AggregateFunctions/AggregateFunctionGroupArray.h | 9 +++++---- .../AggregateFunctionGroupUniqArray.cpp | 2 +- .../AggregateFunctions/AggregateFunctionGroupUniqArray.h | 2 +- dbms/src/AggregateFunctions/AggregateFunctionHistogram.h | 2 +- dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h | 2 +- dbms/src/AggregateFunctions/AggregateFunctionQuantile.h | 2 +- dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp | 4 ++-- dbms/src/AggregateFunctions/AggregateFunctionTopK.h | 2 +- dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp | 2 +- dbms/src/AggregateFunctions/AggregateFunctionUniq.h | 2 +- 13 files changed, 23 insertions(+), 21 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h b/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h index 42649be78fd..9f5d5b69fbd 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionArgMinMax.h @@ -37,7 +37,7 @@ private: public: AggregateFunctionArgMinMax(const DataTypePtr & type_res, const DataTypePtr & type_val) : IAggregateFunctionDataHelper>({type_res, type_val}, {}), - type_res(argument_types[0]), type_val(argument_types[1]) + type_res(this->argument_types[0]), type_val(this->argument_types[1]) { if (!type_val->isComparable()) throw Exception("Illegal type " + type_val->getName() + " of second argument of aggregate function " + getName() diff --git a/dbms/src/AggregateFunctions/AggregateFunctionAvg.cpp b/dbms/src/AggregateFunctions/AggregateFunctionAvg.cpp index 565f1f0c335..1886637629f 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionAvg.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionAvg.cpp @@ -27,9 +27,9 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const AggregateFunctionPtr res; DataTypePtr data_type = argument_types[0]; if (isDecimal(data_type)) - res.reset(createWithDecimalType(*data_type, *data_type)); + res.reset(createWithDecimalType(*data_type, *data_type, argument_types)); else - res.reset(createWithNumericType(*data_type)); + res.reset(createWithNumericType(*data_type, argument_types)); if (!res) throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, diff --git a/dbms/src/AggregateFunctions/AggregateFunctionAvg.h b/dbms/src/AggregateFunctions/AggregateFunctionAvg.h index 98604f76742..d34420efe28 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionAvg.h @@ -49,13 +49,14 @@ public: using ColVecResult = std::conditional_t, ColumnDecimal, ColumnVector>; /// ctor for native types - AggregateFunctionAvg() - : scale(0) + AggregateFunctionAvg(const DataTypes & argument_types) + : IAggregateFunctionDataHelper>(argument_types, {}) + , scale(0) {} /// ctor for Decimals - AggregateFunctionAvg(const IDataType & data_type) - : IAggregateFunctionDataHelper>({data_type}, {}) + AggregateFunctionAvg(const IDataType & data_type, const DataTypes & argument_types) + : IAggregateFunctionDataHelper>(argument_types, {}) , scale(getDecimalScale(data_type)) {} diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h index c496e90844d..d732d65ecf8 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -53,8 +53,8 @@ class GroupArrayNumericImpl final public: explicit GroupArrayNumericImpl(const DataTypePtr & data_type_, UInt64 max_elems_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper, GroupArrayNumericImpl>({data_type}, {}) - , data_type(argument_types[0]), max_elems(max_elems_) {} + : IAggregateFunctionDataHelper, GroupArrayNumericImpl>({data_type_}, {}) + , data_type(this->argument_types[0]), max_elems(max_elems_) {} String getName() const override { return "groupArray"; } @@ -249,12 +249,13 @@ class GroupArrayGeneralListImpl final static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } - DataTypePtr data_type; + DataTypePtr & data_type; UInt64 max_elems; public: GroupArrayGeneralListImpl(const DataTypePtr & data_type, UInt64 max_elems_ = std::numeric_limits::max()) - : data_type(data_type), max_elems(max_elems_) {} + : IAggregateFunctionDataHelper, GroupArrayGeneralListImpl>({data_type}, {}) + , data_type(this->argument_types[0]), max_elems(max_elems_) {} String getName() const override { return "groupArray"; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp index f80a45afaa9..7a99709c33e 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp @@ -48,7 +48,7 @@ AggregateFunctionPtr createAggregateFunctionGroupUniqArray(const std::string & n assertNoParameters(name, parameters); assertUnary(name, argument_types); - AggregateFunctionPtr res(createWithNumericType(*argument_types[0])); + AggregateFunctionPtr res(createWithNumericType(*argument_types[0], argument_types[0])); if (!res) res = AggregateFunctionPtr(createWithExtraTypes(argument_types[0])); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index c0ef1fe0fa8..f2ae9e77438 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -129,7 +129,7 @@ class AggreagteFunctionGroupUniqArrayGeneric public: AggreagteFunctionGroupUniqArrayGeneric(const DataTypePtr & input_data_type) : IAggregateFunctionDataHelper>({input_data_type}, {}) - , input_data_type(argument_types[0]) {} + , input_data_type(this->argument_types[0]) {} String getName() const override { return "groupUniqArray"; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h index 60385f4788a..2e2c979f1d0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -304,7 +304,7 @@ private: const UInt32 max_bins; public: - AggregateFunctionHistogram(const DataTypes & arguments, const Array & params, UInt32 max_bins) + AggregateFunctionHistogram(UInt32 max_bins, const DataTypes & arguments, const Array & params) : IAggregateFunctionDataHelper>(arguments, params) , max_bins(max_bins) { diff --git a/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 426ee8ee479..de661f81115 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -681,7 +681,7 @@ private: public: AggregateFunctionsSingleValue(const DataTypePtr & type) : IAggregateFunctionDataHelper>({type}, {}) - , type(argument_types[0]) + , type(this->argument_types[0]) { if (StringRef(Data::name()) == StringRef("min") || StringRef(Data::name()) == StringRef("max")) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h index a87f520d395..399b7f993d0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -78,7 +78,7 @@ private: public: AggregateFunctionQuantile(const DataTypePtr & argument_type, const Array & params) : IAggregateFunctionDataHelper>({argument_type}, params) - , levels(params, returns_many), level(levels.levels[0]), argument_type(argument_types[0]) + , levels(params, returns_many), level(levels.levels[0]), argument_type(this->argument_types[0]) { if (!returns_many && levels.size() > 1) throw Exception("Aggregate function " + getName() + " require one parameter or less", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp index 5a10ae62324..3c94b0c3705 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionSumMap.cpp @@ -82,7 +82,7 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con AggregateFunctionPtr res(createWithNumericBasedType(*keys_type, keys_type, values_types, arguments)); if (!res) - res.reset(createWithDecimalType(*keys_type, keys_type, values_types)); + res.reset(createWithDecimalType(*keys_type, keys_type, values_types, arguments)); if (!res) throw Exception("Illegal type of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -105,7 +105,7 @@ AggregateFunctionPtr createAggregateFunctionSumMapFiltered(const std::string & n AggregateFunctionPtr res(createWithNumericBasedType(*keys_type, keys_type, values_types, keys_to_keep, arguments, params)); if (!res) - res.reset(createWithDecimalType(*keys_type, keys_type, values_types, keys_to_keep)); + res.reset(createWithDecimalType(*keys_type, keys_type, values_types, keys_to_keep, arguments, params)); if (!res) throw Exception("Illegal type of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h index 846a3e2b2a1..340b5f14e5a 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionTopK.h @@ -145,7 +145,7 @@ public: AggregateFunctionTopKGeneric( UInt64 threshold, const DataTypePtr & input_data_type, const Array & params) : IAggregateFunctionDataHelper>({input_data_type}, params) - , threshold(threshold), reserved(TOP_K_LOAD_FACTOR * threshold), input_data_type(argument_types[0]) {} + , threshold(threshold), reserved(TOP_K_LOAD_FACTOR * threshold), input_data_type(this->argument_types[0]) {} String getName() const override { return is_weighted ? "topKWeighted" : "topK"; } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp b/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp index eaf021d8735..86456af0f9e 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -99,7 +99,7 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const else if (which.isDateTime()) return std::make_shared>>(argument_types); else if (which.isStringOrFixedString()) - return std::make_shared>>(); + return std::make_shared>>(argument_types); else if (which.isUUID()) return std::make_shared>>(argument_types); else if (which.isTuple()) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h index 56a855aabb9..aea227a5d69 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionUniq.h @@ -260,7 +260,7 @@ private: public: AggregateFunctionUniqVariadic(const DataTypes & arguments) - : IAggregateFunctionDataHelper>(arguments) + : IAggregateFunctionDataHelper>(arguments, {}) { if (argument_is_tuple) num_args = typeid_cast(*arguments[0]).getElements().size(); From a4441bfba245eeb7a1a54309b191591144be5a1e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Feb 2019 13:09:03 +0300 Subject: [PATCH 15/40] Fix tests. --- dbms/src/Columns/ColumnAggregateFunction.cpp | 7 ++++++- dbms/src/DataTypes/DataTypeAggregateFunction.cpp | 5 +++-- dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql | 3 ++- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index 23abee39530..176b685548c 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -349,7 +349,12 @@ static void pushBackAndCreateState(ColumnAggregateFunction::Container & data, Ar void ColumnAggregateFunction::insert(const Field & x) { String type_string = getTypeString(); - auto & field_name = x.get().name; + + if (x.getType() != Field::Types::AggregateFunctionState) + throw Exception(String("Inserting field of type ") + x.getTypeName() + " into ColumnAggregateFunction. " + "Expected " + Field::Types::toString(Field::Types::AggregateFunctionState), ErrorCodes::LOGICAL_ERROR); + + auto & field_name = x.get().name; if (type_string != field_name) throw Exception("Cannot insert filed with type " + field_name + " into column with type " + type_string, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp index 33c6ba6a84f..04a06eb98f2 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp @@ -264,7 +264,8 @@ MutableColumnPtr DataTypeAggregateFunction::createColumn() const /// Create empty state Field DataTypeAggregateFunction::getDefault() const { - Field field = String(); + Field field = AggregateFunctionStateData(); + field.get().name = getName(); AlignedBuffer place_buffer(function->sizeOfData(), function->alignOfData()); AggregateDataPtr place = place_buffer.data(); @@ -273,7 +274,7 @@ Field DataTypeAggregateFunction::getDefault() const try { - WriteBufferFromString buffer_from_field(field.get()); + WriteBufferFromString buffer_from_field(field.get().data); function->serialize(place, buffer_from_field); } catch (...) diff --git a/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql b/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql index 5ebb524dc53..35febd5a75b 100644 --- a/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql +++ b/dbms/tests/queries/0_stateless/00205_scalar_subqueries.sql @@ -5,4 +5,5 @@ SELECT toUInt64((SELECT 9)) IN (SELECT number FROM system.numbers LIMIT 10); SELECT (SELECT toDate('2015-01-02')) = toDate('2015-01-02'), 'Hello' = (SELECT 'Hello'); SELECT (SELECT toDate('2015-01-02'), 'Hello'); SELECT (SELECT toDate('2015-01-02'), 'Hello') AS x, x, identity((SELECT 1)), identity((SELECT 1) AS y); -SELECT (SELECT uniqState('')); -- { serverError 125 } +-- SELECT (SELECT uniqState('')); + From ef6b20030cc6f38e3d0ba340a569d934f246f616 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 12 Feb 2019 14:15:03 +0300 Subject: [PATCH 16/40] Fix compilation on Mac. --- dbms/src/Columns/ColumnDecimal.h | 2 +- dbms/src/Columns/ColumnVector.h | 2 +- dbms/src/Columns/IColumnUnique.h | 6 +++--- dbms/src/Storages/IStorage.h | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/dbms/src/Columns/ColumnDecimal.h b/dbms/src/Columns/ColumnDecimal.h index 2bdaaf8d536..308d237c3a9 100644 --- a/dbms/src/Columns/ColumnDecimal.h +++ b/dbms/src/Columns/ColumnDecimal.h @@ -166,7 +166,7 @@ template template ColumnPtr ColumnDecimal::indexImpl(const PaddedPODArray & indexes, UInt64 limit) const { - size_t size = indexes.size(); + UInt64 size = indexes.size(); if (limit == 0) limit = size; diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 7159c25aa59..da3e62aac03 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -275,7 +275,7 @@ template template ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, UInt64 limit) const { - size_t size = indexes.size(); + UInt64 size = indexes.size(); if (limit == 0) limit = size; diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h index f3567fb8760..fbb2e5625f7 100644 --- a/dbms/src/Columns/IColumnUnique.h +++ b/dbms/src/Columns/IColumnUnique.h @@ -99,7 +99,7 @@ public: throw Exception("Method deserializeAndInsertFromArena is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } - ColumnPtr index(const IColumn &, size_t) const override + ColumnPtr index(const IColumn &, UInt64) const override { throw Exception("Method index is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } @@ -114,7 +114,7 @@ public: throw Exception("Method filter is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } - ColumnPtr permute(const IColumn::Permutation &, size_t) const override + ColumnPtr permute(const IColumn::Permutation &, UInt64) const override { throw Exception("Method permute is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } @@ -124,7 +124,7 @@ public: throw Exception("Method replicate is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } - void getPermutation(bool, size_t, int, IColumn::Permutation &) const override + void getPermutation(bool, UInt64, int, IColumn::Permutation &) const override { throw Exception("Method getPermutation is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 7397d285b80..53adf573c52 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -185,7 +185,7 @@ public: const SelectQueryInfo & /*query_info*/, const Context & /*context*/, QueryProcessingStage::Enum /*processed_stage*/, - size_t /*max_block_size*/, + UInt64 /*max_block_size*/, unsigned /*num_streams*/) { throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); From 558ebbcc31c3b4cf7fb7220c184f3510001ee22e Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 12 Feb 2019 14:17:46 +0300 Subject: [PATCH 17/40] New library clickhouse_storage_kafka --- dbms/CMakeLists.txt | 9 +-------- dbms/src/Storages/CMakeLists.txt | 9 +++++---- dbms/src/Storages/Kafka/CMakeLists.txt | 9 +++++++++ 3 files changed, 15 insertions(+), 12 deletions(-) create mode 100644 dbms/src/Storages/Kafka/CMakeLists.txt diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 4f31b16cbb0..65a99cede0d 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -102,9 +102,6 @@ add_headers_and_sources(dbms src/Interpreters/ClusterProxy) add_headers_and_sources(dbms src/Columns) add_headers_and_sources(dbms src/Storages) add_headers_and_sources(dbms src/Storages/Distributed) -if(USE_RDKAFKA) - add_headers_and_sources(dbms src/Storages/Kafka) -endif() add_headers_and_sources(dbms src/Storages/MergeTree) add_headers_and_sources(dbms src/Client) add_headers_and_sources(dbms src/Formats) @@ -297,11 +294,7 @@ if (USE_CAPNP) endif () if (USE_RDKAFKA) - target_link_libraries (dbms PRIVATE ${RDKAFKA_LIBRARY}) - target_link_libraries (dbms PRIVATE ${CPPKAFKA_LIBRARY}) - if (NOT USE_INTERNAL_RDKAFKA_LIBRARY) - target_include_directories (dbms SYSTEM BEFORE PRIVATE ${RDKAFKA_INCLUDE_DIR}) - endif () + target_link_libraries (dbms PRIVATE clickhouse_storage_kafka) endif () target_link_libraries(dbms PRIVATE ${OPENSSL_CRYPTO_LIBRARY} Threads::Threads) diff --git a/dbms/src/Storages/CMakeLists.txt b/dbms/src/Storages/CMakeLists.txt index 617e866a012..236d4d32524 100644 --- a/dbms/src/Storages/CMakeLists.txt +++ b/dbms/src/Storages/CMakeLists.txt @@ -1,5 +1,6 @@ -add_subdirectory (System) +add_subdirectory(System) +add_subdirectory(Kafka) -if (ENABLE_TESTS) - add_subdirectory (tests) -endif () +if(ENABLE_TESTS) + add_subdirectory(tests) +endif() diff --git a/dbms/src/Storages/Kafka/CMakeLists.txt b/dbms/src/Storages/Kafka/CMakeLists.txt new file mode 100644 index 00000000000..c764a1b7fd5 --- /dev/null +++ b/dbms/src/Storages/Kafka/CMakeLists.txt @@ -0,0 +1,9 @@ +if(USE_RDKAFKA) + include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) + add_headers_and_sources(clickhouse_storage_kafka .) + add_library(clickhouse_storage_kafka ${LINK_MODE} ${clickhouse_storage_kafka_sources}) + target_link_libraries(clickhouse_storage_kafka PRIVATE clickhouse_common_io ${RDKAFKA_LIBRARY} ${CPPKAFKA_LIBRARY}) + if(NOT USE_INTERNAL_RDKAFKA_LIBRARY) + target_include_directories(clickhouse_storage_kafka SYSTEM BEFORE PRIVATE ${RDKAFKA_INCLUDE_DIR}) + endif() +endif() From 75a2d649d09f56707fcbe7f120716bd8b00feec4 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 12 Feb 2019 14:59:34 +0300 Subject: [PATCH 18/40] DOCAPI-4180: merge_tree* settings. EN review. RU translation (#4358) --- docs/en/operations/requirements.md | 2 +- docs/en/operations/settings/settings.md | 110 ++++++------ .../ru/operations/server_settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 158 ++++++++++++------ docs/toc_ru.yml | 2 +- 5 files changed, 165 insertions(+), 109 deletions(-) diff --git a/docs/en/operations/requirements.md b/docs/en/operations/requirements.md index 62150c0f456..cfe6e28a5fe 100644 --- a/docs/en/operations/requirements.md +++ b/docs/en/operations/requirements.md @@ -2,7 +2,7 @@ ## CPU -For installation from prebuilt deb packages, use a CPU with x86_64 architecture and support for SSE 4.2 instructions. To run ClickHouse with processors than do not support SSE 4.2 or have AArch64 or PowerPC64LE architecture, you should build ClickHouse from sources. +For installation from prebuilt deb packages, use a CPU with x86_64 architecture and support for SSE 4.2 instructions. To run ClickHouse with processors that do not support SSE 4.2 or have AArch64 or PowerPC64LE architecture, you should build ClickHouse from sources. ClickHouse implements parallel data processing and uses all the hardware resources available. When choosing a processor, take into account that ClickHouse works more efficiently at configurations with a large number of cores but a lower clock rate than at configurations with fewer cores and a higher clock rate. For example, 16 cores with 2600 MHz is preferable to 8 cores with 3600 MHz. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 836a13baeb0..6b81a9f539a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -87,23 +87,22 @@ For INSERT queries, specifies that the server need to send metadata about column ## join_default_strictness -Sets default strictness for [JOIN clause](../../query_language/select.md). +Sets default strictness for [JOIN clauses](../../query_language/select.md). **Possible values** -- `ALL` — If the right table has several matching rows, the data will be multiplied by the number of these rows. It is a normal `JOIN` behavior from standard SQL. +- `ALL` — If the right table has several matching rows, the data is multiplied by the number of these rows. This is the normal `JOIN` behavior from standard SQL. - `ANY` — If the right table has several matching rows, only the first one found is joined. If the right table has only one matching row, the results of `ANY` and `ALL` are the same. -- `Empty string` — If `ALL` or `ANY` not specified in query, ClickHouse throws exception. +- `Empty string` — If `ALL` or `ANY` is not specified in the query, ClickHouse throws an exception. -**Default value** +**Default value**: `ALL` -`ALL` ## max_block_size -In ClickHouse, data is processed by blocks (sets of column parts). The internal processing cycles for a single block are efficient enough, but there are noticeable expenditures on each block. `max_block_size` is a recommendation for what size of block (in number of rows) to load from tables. The block size shouldn't be too small, so that the expenditures on each block are still noticeable, but not too large, so that the query with LIMIT that is completed after the first block is processed quickly, so that too much memory isn't consumed when extracting a large number of columns in multiple threads, and so that at least some cache locality is preserved. +In ClickHouse, data is processed by blocks (sets of column parts). The internal processing cycles for a single block are efficient enough, but there are noticeable expenditures on each block. The `max_block_size` setting is a recommendation for what size of block (in number of rows) to load from tables. The block size shouldn't be too small, so that the expenditures on each block are still noticeable, but not too large, so that the query with LIMIT that is completed after the first block is processed quickly. The goal is to avoid consuming too much memory when extracting a large number of columns in multiple threads, and to preserve at least some cache locality. -By default, 65,536. +Default value: 65,536. Blocks the size of `max_block_size` are not always loaded from the table. If it is obvious that less data needs to be retrieved, a smaller block is processed. @@ -115,44 +114,44 @@ By default: 1,000,000. It only works when reading from MergeTree engines. ## merge_tree_uniform_read_distribution {#setting-merge_tree_uniform_read_distribution} -When reading from [MergeTree*](../table_engines/mergetree.md) tables, ClickHouse uses several threads. This setting turns on/off the uniform distribution of reading tasks over the working threads. The algorithm of the uniform distribution aims to make execution time for all the threads approximately equal in a `SELECT` query. +ClickHouse uses multiple threads when reading from [MergeTree*](../table_engines/mergetree.md) tables. This setting turns on/off the uniform distribution of reading tasks over the working threads. The algorithm of the uniform distribution aims to make execution time for all the threads approximately equal in a `SELECT` query. **Possible values** -- 0 — Uniform read distribution turned off. -- 1 — Uniform read distribution turned on. +- 0 — Do not use uniform read distribution. +- 1 — Use uniform read distribution. -**Default value** — 1. +**Default value**: 1. ## merge_tree_min_rows_for_concurrent_read {#setting-merge_tree_min_rows_for_concurrent_read} -If a number of rows to be read from a file of [MergeTree*](../table_engines/mergetree.md) table exceeds `merge_tree_min_rows_for_concurrent_read` then ClickHouse tries to perform a concurrent reading from this file by several threads. +If the number of rows to be read from a file of a [MergeTree*](../table_engines/mergetree.md) table exceeds `merge_tree_min_rows_for_concurrent_read` then ClickHouse tries to perform a concurrent reading from this file on several threads. **Possible values** Any positive integer. -**Default value** — 163840. +**Default value**: 163840. ## merge_tree_min_rows_for_seek {#setting-merge_tree_min_rows_for_seek} -If the distance between two data blocks to be read in one file less than `merge_tree_min_rows_for_seek` rows, then ClickHouse does not seek through the file, it reads the data sequentially. +If the distance between two data blocks to be read in one file is less than `merge_tree_min_rows_for_seek` rows, then ClickHouse does not seek through the file, but reads the data sequentially. **Possible values** Any positive integer. -**Default value** — 0. +**Default value**: 0. ## merge_tree_coarse_index_granularity {#setting-merge_tree_coarse_index_granularity} -When searching data, ClickHouse checks the data marks in the index file. If ClickHouse finds that required keys are in some range, it divides this range for `merge_tree_coarse_index_granularity` subranges and searches the required keys there recursively. +When searching data, ClickHouse checks the data marks in the index file. If ClickHouse finds that required keys are in some range, it divides this range into `merge_tree_coarse_index_granularity` subranges and searches the required keys there recursively. **Possible values** Any positive even integer. -**Default value** — 8. +**Default value**: 8. ## merge_tree_max_rows_to_use_cache {#setting-merge_tree_max_rows_to_use_cache} @@ -162,7 +161,7 @@ If ClickHouse should read more than `merge_tree_max_rows_to_use_cache` rows in o Any positive integer. -**Default value** — 1048576. +**Default value**: 1048576. ## log_queries @@ -182,9 +181,9 @@ For example, for an INSERT via the HTTP interface, the server parses the data fo But when using clickhouse-client, the client parses the data itself, and the 'max_insert_block_size' setting on the server doesn't affect the size of the inserted blocks. The setting also doesn't have a purpose when using INSERT SELECT, since data is inserted using the same blocks that are formed after SELECT. -By default, it is 1,048,576. +Default value: 1,048,576. -This is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allows sorting more data in RAM. +The default is slightly more than `max_block_size`. The reason for this is because certain table engines (`*MergeTree`) form a data part on the disk for each inserted block, which is a fairly large entity. Similarly, `*MergeTree` tables sort data during insertion, and a large enough block size allows sorting more data in RAM. ## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} @@ -198,18 +197,16 @@ Used when performing `SELECT` from a distributed table that points to replicated ## max_threads {#settings-max_threads} -The maximum number of query processing threads - -- excluding threads for retrieving data from remote servers (see the 'max_distributed_connections' parameter). +The maximum number of query processing threads, excluding threads for retrieving data from remote servers (see the 'max_distributed_connections' parameter). This parameter applies to threads that perform the same stages of the query processing pipeline in parallel. -For example, if reading from a table, evaluating expressions with functions, filtering with WHERE and pre-aggregating for GROUP BY can all be done in parallel using at least 'max_threads' number of threads, then 'max_threads' are used. +For example, when reading from a table, if it is possible to evaluate expressions with functions, filter with WHERE and pre-aggregate for GROUP BY in parallel using at least 'max_threads' number of threads, then 'max_threads' are used. -By default, 2. +Default value: 2. If less than one SELECT query is normally run on a server at a time, set this parameter to a value slightly less than the actual number of processor cores. -For queries that are completed quickly because of a LIMIT, you can set a lower 'max_threads'. For example, if the necessary number of entries are located in every block and max_threads = 8, 8 blocks are retrieved, although it would have been enough to read just one. +For queries that are completed quickly because of a LIMIT, you can set a lower 'max_threads'. For example, if the necessary number of entries are located in every block and max_threads = 8, then 8 blocks are retrieved, although it would have been enough to read just one. The smaller the `max_threads` value, the less memory is consumed. @@ -217,7 +214,7 @@ The smaller the `max_threads` value, the less memory is consumed. The maximum size of blocks of uncompressed data before compressing for writing to a table. By default, 1,048,576 (1 MiB). If the size is reduced, the compression rate is significantly reduced, the compression and decompression speed increases slightly due to cache locality, and memory consumption is reduced. There usually isn't any reason to change this setting. -Don't confuse blocks for compression (a chunk of memory consisting of bytes) and blocks for query processing (a set of rows from a table). +Don't confuse blocks for compression (a chunk of memory consisting of bytes) with blocks for query processing (a set of rows from a table). ## min_compress_block_size @@ -238,31 +235,31 @@ There usually isn't any reason to change this setting. The maximum part of a query that can be taken to RAM for parsing with the SQL parser. The INSERT query also contains data for INSERT that is processed by a separate stream parser (that consumes O(1) RAM), which is not included in this restriction. -The default is 256 KiB. +Default value: 256 KiB. ## interactive_delay The interval in microseconds for checking whether request execution has been canceled and sending the progress. -By default, 100,000 (check for canceling and send progress ten times per second). +Default value: 100,000 (checks for canceling and sends the progress ten times per second). ## connect_timeout, receive_timeout, send_timeout Timeouts in seconds on the socket used for communicating with the client. -By default, 10, 300, 300. +Default value: 10, 300, 300. ## poll_interval Lock in a wait loop for the specified number of seconds. -By default, 10. +Default value: 10. ## max_distributed_connections The maximum number of simultaneous connections with remote servers for distributed processing of a single query to a single Distributed table. We recommend setting a value no less than the number of servers in the cluster. -By default, 1024. +Default value: 1024. The following parameters are only used when creating Distributed tables (and when launching a server), so there is no reason to change them at runtime. @@ -270,20 +267,20 @@ The following parameters are only used when creating Distributed tables (and whe The maximum number of simultaneous connections with remote servers for distributed processing of all queries to a single Distributed table. We recommend setting a value no less than the number of servers in the cluster. -By default, 1024. +Default value: 1024. ## connect_timeout_with_failover_ms The timeout in milliseconds for connecting to a remote server for a Distributed table engine, if the 'shard' and 'replica' sections are used in the cluster definition. If unsuccessful, several attempts are made to connect to various replicas. -By default, 50. +Default value: 50. ## connections_with_failover_max_tries -The maximum number of connection attempts with each replica, for the Distributed table engine. +The maximum number of connection attempts with each replica for the Distributed table engine. -By default, 3. +Default value: 3. ## extremes @@ -292,10 +289,10 @@ For more information, see the section "Extreme values". ## use_uncompressed_cache {#setting-use_uncompressed_cache} -Whether to use a cache of uncompressed blocks. Accepts 0 or 1. By default, 1 (enabled). -The uncompressed cache (only for tables in the MergeTree family) allows significantly reducing latency and increasing throughput when working with a large number of short queries. Enable this setting for users who send frequent short requests. Also pay attention to the [uncompressed_cache_size](../server_settings/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed; the least-used data is automatically deleted. +Whether to use a cache of uncompressed blocks. Accepts 0 or 1. By default, 0 (disabled). +Using the uncompressed cache (only for tables in the MergeTree family) can significantly reduce latency and increase throughput when working with a large number of short queries. Enable this setting for users who send frequent short requests. Also pay attention to the [uncompressed_cache_size](../server_settings/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. -For queries that read at least a somewhat large volume of data (one million rows or more), the uncompressed cache is disabled automatically in order to save space for truly small queries. So you can keep the 'use_uncompressed_cache' setting always set to 1. +For queries that read at least a somewhat large volume of data (one million rows or more), the uncompressed cache is disabled automatically in order to save space for truly small queries. This means that you can keep the 'use_uncompressed_cache' setting always set to 1. ## replace_running_query @@ -336,7 +333,7 @@ Disadvantages: Server proximity is not accounted for; if the replicas have diffe The number of errors is counted for each replica. Every 5 minutes, the number of errors is integrally divided by 2. Thus, the number of errors is calculated for a recent time with exponential smoothing. If there is one replica with a minimal number of errors (i.e. errors occurred recently on the other replicas), the query is sent to it. If there are multiple replicas with the same minimal number of errors, the query is sent to the replica with a host name that is most similar to the server's host name in the config file (for the number of different characters in identical positions, up to the minimum length of both host names). For instance, example01-01-1 and example01-01-2.yandex.ru are different in one position, while example01-01-1 and example01-02-2 differ in two places. -This method might seem a little stupid, but it doesn't use external data about network topology, and it doesn't compare IP addresses, which would be complicated for our IPv6 addresses. +This method might seem primitive, but it doesn't require external data about network topology, and it doesn't compare IP addresses, which would be complicated for our IPv6 addresses. Thus, if there are equivalent replicas, the closest one by name is preferred. We can also assume that when sending a query to the same server, in the absence of failures, a distributed query will also go to the same servers. So even if different data is placed on the replicas, the query will return mostly the same results. @@ -372,8 +369,8 @@ If this portion of the pipeline was compiled, the query may run faster due to de ## min_count_to_compile How many times to potentially use a compiled chunk of code before running compilation. By default, 3. -If the value is zero, then compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. This can be used for testing; otherwise, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. -If the value is 1 or more, compilation occurs asynchronously in a separate thread. The result will be used as soon as it is ready, including by queries that are currently running. +For testing, the value can be set to 0: compilation runs synchronously and the query waits for the end of the compilation process before continuing execution. For all other cases, use values ​​starting with 1. Compilation normally takes about 5-10 seconds. +If the value is 1 or more, compilation occurs asynchronously in a separate thread. The result will be used as soon as it is ready, including queries that are currently running. Compiled code is required for each different combination of aggregate functions used in the query and the type of keys in the GROUP BY clause. The results of compilation are saved in the build directory in the form of .so files. There is no restriction on the number of compilation results, since they don't use very much space. Old results will be used after server restarts, except in the case of a server upgrade – in this case, the old results are deleted. @@ -397,14 +394,14 @@ Affects the behavior of [JOIN](../../query_language/select.md). With `join_use_nulls=1,` `JOIN` behaves like in standard SQL, i.e. if empty cells appear when merging, the type of the corresponding field is converted to [Nullable](../../data_types/nullable.md#data_type-nullable), and empty cells are filled with [NULL](../../query_language/syntax.md). -## insert_quorum +## insert_quorum {#settings-insert_quorum} Enables quorum writes. - If `insert_quorum < 2`, the quorum writes are disabled. - If `insert_quorum >= 2`, the quorum writes are enabled. -The default value is 0. +Default value: 0. **Quorum writes** @@ -412,7 +409,7 @@ The default value is 0. All the replicas in the quorum are consistent, i.e., they contain data from all previous `INSERT` queries. The `INSERT` sequence is linearized. -When reading the data written from the `insert_quorum`, you can use the [select_sequential_consistency](#select-sequential-consistency) option. +When reading the data written from the `insert_quorum`, you can use the [select_sequential_consistency](#settings-select_sequential_consistency) option. **ClickHouse generates an exception** @@ -421,35 +418,36 @@ When reading the data written from the `insert_quorum`, you can use the [select_ **See also the following parameters:** -- [insert_quorum_timeout](#insert-quorum-timeout) -- [select_sequential_consistency](#select-sequential-consistency) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## insert_quorum_timeout +## insert_quorum_timeout {#settings-insert_quorum_timeout} Quorum write timeout in seconds. If the timeout has passed and no write has taken place yet, ClickHouse will generate an exception and the client must repeat the query to write the same block to the same or any other replica. -By default, 60 seconds. +Default value: 60 seconds. **See also the following parameters:** -- [insert_quorum](#insert-quorum) -- [select_sequential_consistency](#select-sequential-consistency) +- [insert_quorum](#settings-insert_quorum) +- [select_sequential_consistency](#settings-select_sequential_consistency) -## select_sequential_consistency +## select_sequential_consistency {#settings-select_sequential_consistency} Enables/disables sequential consistency for `SELECT` queries: -- 0 — disabled. The default value is 0. -- 1 — enabled. +- 0 — Disabled. +- 1 — Enabled. +Default value: 0. When sequential consistency is enabled, ClickHouse allows the client to execute the `SELECT` query only for those replicas that contain data from all previous `INSERT` queries executed with `insert_quorum`. If the client refers to a partial replica, ClickHouse will generate an exception. The SELECT query will not include data that has not yet been written to the quorum of replicas. See also the following parameters: -- [insert_quorum](#insert-quorum) -- [insert_quorum_timeout](#insert-quorum-timeout) +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index a9e904c7dd3..faee490222e 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -627,7 +627,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat ``` -## uncompressed_cache_size +## uncompressed_cache_size {#server-settings-uncompressed_cache_size} Размер кеша (в байтах) для несжатых данных, используемых движками таблиц семейства [MergeTree](../../operations/table_engines/mergetree.md). diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 7f3cc3c9c77..3c6b0087f16 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1,6 +1,5 @@ # Настройки - ## distributed_product_mode Изменяет поведение [распределенных подзапросов](../../query_language/select.md). @@ -9,19 +8,18 @@ ClickHouse применяет настройку в тех случаях, ко Условия применения: -- Только подзапросы для IN, JOIN. -- Только если в секции FROM используется распределённая таблица, содержащая более одного шарда. -- Если подзапрос касается распределенной таблицы, содержащей более одного шарда, -- Не используется в случае табличной функции [remote](../../query_language/table_functions/remote.md). +- Только подзапросы для IN, JOIN. +- Только если в секции FROM используется распределённая таблица, содержащая более одного шарда. +- Если подзапрос касается распределенной таблицы, содержащей более одного шарда, +- Не используется в случае табличной функции [remote](../../query_language/table_functions/remote.md). Возможные значения: -- `deny` - (по умолчанию) запрещает использование таких подзапросов (При попытке использование вернет исключение "Double-distributed IN/JOIN subqueries is denied"); -- `local` - заменит базу данных и таблицу в подзапросе на локальные для конечного сервера (шарда), оставив обычный `IN` / `JOIN`; -- `global` - заменит запрос `IN` / `JOIN` на `GLOBAL IN` / `GLOBAL JOIN`; +- `deny` — значение по умолчанию. Запрещает использование таких подзапросов (При попытке использование вернет исключение "Double-distributed IN/JOIN subqueries is denied"); +- `local` - заменяет базу данных и таблицу в подзапросе на локальные для конечного сервера (шарда), оставив обычный `IN` / `JOIN.` +- `global` - заменяет запрос `IN` / `JOIN` на `GLOBAL IN` / `GLOBAL JOIN.` - `allow` - разрешает использование таких подзапросов. - ## fallback_to_stale_replicas_for_distributed_queries Форсирует запрос в устаревшую реплику в случае, если актуальные данные недоступны. Смотрите "[Репликация](../../operations/table_engines/replication.md)". @@ -40,7 +38,6 @@ ClickHouse применяет настройку в тех случаях, ко При `force_index_by_date=1` ClickHouse проверяет, есть ли в запросе условие на ключ даты, которое может использоваться для отсечения диапазонов данных. Если подходящего условия нет - кидается исключение. При этом не проверяется, действительно ли условие уменьшает объём данных для чтения. Например, условие `Date != '2000-01-01'` подходит даже в том случае, когда соответствует всем данным в таблице (т.е. для выполнения запроса требуется full scan). Подробнее про диапазоны данных в таблицах MergeTree читайте в разделе "[MergeTree](../../operations/table_engines/mergetree.md)". - ## force_primary_key Запрещает выполнение запросов, если использовать индекс по первичному ключу невозможно. @@ -49,7 +46,6 @@ ClickHouse применяет настройку в тех случаях, ко При `force_primary_key=1` ClickHouse проверяет, есть ли в запросе условие на первичный ключ, которое может использоваться для отсечения диапазонов данных. Если подходящего условия нет - кидается исключение. При этом не проверяется, действительно ли условие уменьшает объём данных для чтения. Подробнее про диапазоны данных в таблицах MergeTree читайте в разделе "[MergeTree](../../operations/table_engines/mergetree.md)". - ## fsync_metadata Включить или отключить fsync при записи .sql файлов. По умолчанию включено. @@ -60,7 +56,7 @@ ClickHouse применяет настройку в тех случаях, ко Устанавливает максимальное количество допустимых ошибок при чтении из текстовых форматов (CSV, TSV и т.п.). -Значение по умолчанию - 0. +Значение по умолчанию: 0. Используйте обязательно в паре с `input_format_allow_errors_ratio`. Для пропуска ошибок, значения обеих настроек должны быть больше 0. @@ -73,7 +69,7 @@ ClickHouse применяет настройку в тех случаях, ко Устанавливает максимальную долю допустимых ошибок при чтении из текстовых форматов (CSV, TSV и т.п.). Доля ошибок задаётся в виде числа с плавающей запятой от 0 до 1. -Значение по умолчанию - 0. +Значение по умолчанию: 0. Используйте обязательно в паре с `input_format_allow_errors_num`. Для пропуска ошибок, значения обеих настроек должны быть больше 0. @@ -81,11 +77,23 @@ ClickHouse применяет настройку в тех случаях, ко В случае превышения `input_format_allow_errors_ratio` ClickHouse генерирует исключение. +## join_default_strictness + +Устанавливает строгость по умолчанию для [JOIN](../../query_language/select.md). + +**Возможные значения** + +- `ALL` — если в правой таблице несколько совпадающих строк, данные умножаются на количество этих строк. Это нормальное поведение `JOIN` как в стандартном SQL. +- `ANY` — если в правой таблице несколько соответствующих строк, то соединяется только первая найденная. Если в "правой" таблице есть не более одной подходящей строки, то результаты `ANY` и `ALL` совпадают. +- `Пустая строка` — если `ALL` или `ANY` не указаны в запросе, то ClickHouse генерирует исключение. + +**Значение по умолчанию**: `ALL` + ## max_block_size -Данные в ClickHouse обрабатываются по блокам (наборам кусочков столбцов). Внутренние циклы обработки одного блока достаточно эффективны, но при этом существуют заметные издержки на каждый блок. `max_block_size` - это рекомендация, какого размера блоки (в количестве строк) загружать из таблицы. Размер блока должен быть не слишком маленьким, чтобы издержки на каждый блок оставались незаметными, и не слишком большим, чтобы запрос с LIMIT-ом, который завершается уже после первого блока, выполнялся быстро; чтобы не использовалось слишком много оперативки при вынимании большого количества столбцов в несколько потоков; чтобы оставалась хоть какая-нибудь кэш-локальность. +Данные в ClickHouse обрабатываются по блокам (наборам кусочков столбцов). Внутренние циклы обработки для одного блока достаточно эффективны, но есть заметные издержки на каждый блок. Настройка `max_block_size` — это рекомендация, какой размер блока (в количестве строк) загружать из таблиц. Размер блока не должен быть слишком маленьким, чтобы затраты на каждый блок были заметны, но не слишком велики, чтобы запрос с LIMIT, который завершается после первого блока, обрабатывался быстро. Цель состоит в том, чтобы не использовалось слишком много оперативки при вынимании большого количества столбцов в несколько потоков; чтобы оставалась хоть какая-нибудь кэш-локальность. -По умолчанию - 65 536. +Значение по умолчанию: 65,536. Из таблицы не всегда загружаются блоки размера `max_block_size`. Если ясно, что нужно прочитать меньше данных, то будет считан блок меньшего размера. @@ -93,14 +101,67 @@ ClickHouse применяет настройку в тех случаях, ко Служит для тех же целей что и `max_block_size`, но задает реккомедуемый размер блоков в байтах, выбирая адаптивное количество строк в блоке. При этом размер блока не может быть более `max_block_size` строк. + Значение по умолчанию: 1,000,000. Работает только при чтении из MergeTree-движков. +Отключена по умолчанию (значение 0). Работает только при чтении из MergeTree-движков. + +## merge_tree_uniform_read_distribution {#setting-merge_tree_uniform_read_distribution} + +При чтении из таблиц [MergeTree*](../table_engines/mergetree.md) ClickHouse использует несколько потоков. Этот параметр включает/выключает равномерное распределение заданий по рабочим потокам. Алгоритм равномерного распределения стремится сделать время выполнения всех потоков примерно равным для одного запроса `SELECT`. + +**Возможные значения** + +- 0 — не использовать равномерное распределение заданий на чтение. +- 1 — использовать равномерное распределение заданий на чтение. + +**Значение по умолчанию**: 1. + +## merge_tree_min_rows_for_concurrent_read {#setting-merge_tree_min_rows_for_concurrent_read} + +Если количество строк, считываемых из файла таблицы [MergeTree*](../table_engines/mergetree.md) превышает `merge_tree_min_rows_for_concurrent_read`, то ClickHouse пытается выполнить одновременное чтение из этого файла в несколько потоков. + +**Возможные значения** + +Любое положительное целое число. + +**Значение по умолчанию**: 163840. + +## merge_tree_min_rows_for_seek {#setting-merge_tree_min_rows_for_seek} + +Если расстояние между двумя блоками данных для чтения в одном файле меньше, чем `merge_tree_min_rows_for_seek` строк, то ClickHouse не перескакивает через блоки, а считывает данные последовательно. + +**Возможные значения** + +Любое положительное целое число. + +**Значение по умолчанию**: 0. + +## merge_tree_coarse_index_granularity {#setting-merge_tree_coarse_index_granularity} + +При поиске данных ClickHouse проверяет засечки данных в файле индекса. Если ClickHouse обнаруживает, что требуемые ключи находятся в некотором диапазоне, он делит этот диапазон на `merge_tree_coarse_index_granularity` поддиапазонов и выполняет в них рекурсивный поиск нужных ключей. + +**Возможные значения** + +Любое положительное целое число. + +**Значение по умолчанию**: 8. + +## merge_tree_max_rows_to_use_cache {#setting-merge_tree_max_rows_to_use_cache} + +Если требуется прочитать более, чем `merge_tree_max_rows_to_use_cache` строк в одном запросе, ClickHouse не используют кэш несжатых блоков. Настройка сервера [uncompressed_cache_size](../server_settings/settings.md#server-settings-uncompressed_cache_size) определяет размер кэша несжатых блоков. + +**Возможные значения** + +Любое положительное целое число. + +**Значение по умолчанию**: 1048576. ## log_queries Установка логгирования запроса. -Запросы, переданные в ClickHouse с этой установкой, логгируются согласно правилам конфигурационного параметра сервера [query_log](../server_settings/settings.md). +Запросы, переданные в ClickHouse с этой установкой, логгируются согласно правилам конфигурационного параметра сервера [query_log](../server_settings/settings.md) . **Пример** : @@ -114,9 +175,9 @@ ClickHouse применяет настройку в тех случаях, ко А при использовании clickhouse-client, клиент сам парсит данные, и настройка max_insert_block_size на сервере не влияет на размер вставляемых блоков. При использовании INSERT SELECT, настройка так же не имеет смысла, так как данные будут вставляться теми блоками, которые вышли после SELECT-а. -По умолчанию - 1 048 576. +Значение по умолчанию: 1,048,576. -Это намного больше, чем `max_block_size`. Это сделано, потому что некоторые движки таблиц (`*MergeTree`) будут на каждый вставляемый блок формировать кусок данных на диске, что является довольно большой сущностью. Также, в таблицах типа `*MergeTree`, данные сортируются при вставке, и достаточно большой размер блока позволяет отсортировать больше данных в оперативке. +Это значение намного больше, чем `max_block_size`. Это сделано, потому что некоторые движки таблиц (`*MergeTree`) будут на каждый вставляемый блок формировать кусок данных на диске, что является довольно большой сущностью. Также, в таблицах типа `*MergeTree`, данные сортируются при вставке, и достаточно большой размер блока позволяет отсортировать больше данных в оперативке. ## max_replica_delay_for_distributed_queries {#settings-max_replica_delay_for_distributed_queries} @@ -130,11 +191,10 @@ ClickHouse применяет настройку в тех случаях, ко ## max_threads {#settings-max_threads} -Максимальное количество потоков обработки запроса -- без учёта потоков для чтения данных с удалённых серверов (смотрите параметр max_distributed_connections). +Максимальное количество потоков обработки запроса без учёта потоков для чтения данных с удалённых серверов (смотрите параметр max_distributed_connections). Этот параметр относится к потокам, которые выполняют параллельно одни стадии конвейера выполнения запроса. -Например, если чтение из таблицы, вычисление выражений с функциями, фильтрацию с помощью WHERE и предварительную агрегацию для GROUP BY можно делать параллельно с использованием как минимум max_threads потоков, то будет использовано max_threads потоков. +Например, при чтении из таблицы, если есть возможность вычислять выражения с функциями, фильтровать с помощью WHERE и предварительно агрегировать для GROUP BY параллельно, используя хотя бы количество потоков max_threads, то используются max_threads. По умолчанию - 2. @@ -152,7 +212,8 @@ ClickHouse применяет настройку в тех случаях, ко ## min_compress_block_size -Для таблиц типа "[MergeTree](../../operations/table_engines/mergetree.md +Для таблиц типа "[MergeTree](../../operations/table_engines/mergetree.md)". В целях уменьшения задержек при обработке запросов, блок сжимается при записи следующей засечки, если его размер не меньше min_compress_block_size. По умолчанию - 65 536. + Реальный размер блока, если несжатых данных меньше max_compress_block_size, будет не меньше этого значения и не меньше объёма данных на одну засечку. Рассмотрим пример. Пусть index_granularity, указанная при создании таблицы - 8192. @@ -168,25 +229,25 @@ ClickHouse применяет настройку в тех случаях, ко Максимальный кусок запроса, который будет считан в оперативку для разбора парсером языка SQL. Запрос INSERT также содержит данные для INSERT-а, которые обрабатываются отдельным, потоковым парсером (расходующим O(1) оперативки), и не учитываются в этом ограничении. -По умолчанию - 256 KiB. +Значение по умолчанию: 256 Кб. ## interactive_delay Интервал в микросекундах для проверки, не запрошена ли остановка выполнения запроса, и отправки прогресса. -По умолчанию - 100 000 (проверять остановку запроса и отправлять прогресс десять раз в секунду). +Значение по умолчанию: 100,000 (проверять остановку запроса и отправлять прогресс десять раз в секунду). ## connect_timeout, receive_timeout, send_timeout Таймауты в секундах на сокет, по которому идёт общение с клиентом. -По умолчанию - 10, 300, 300. +Значение по умолчанию: 10, 300, 300. ## poll_interval Блокироваться в цикле ожидания запроса в сервере на указанное количество секунд. -По умолчанию - 10. +Значение по умолчанию: 10. ## max_distributed_connections @@ -207,30 +268,29 @@ ClickHouse применяет настройку в тех случаях, ко Таймаут в миллисекундах на соединение с удалённым сервером, для движка таблиц Distributed, если используются секции shard и replica в описании кластера. В случае неуспеха, делается несколько попыток соединений с разными репликами. -По умолчанию - 50. +Значение по умолчанию: 50. ## connections_with_failover_max_tries Максимальное количество попыток соединения с каждой репликой, для движка таблиц Distributed. -По умолчанию - 3. +Значение по умолчанию: 3. ## extremes Считать ли экстремальные значения (минимумы и максимумы по столбцам результата запроса). Принимает 0 или 1. По умолчанию - 0 (выключено). Подробнее смотрите раздел "Экстремальные значения". +## use_uncompressed_cache {#setting-use_uncompressed_cache} -## use_uncompressed_cache +Использовать ли кэш разжатых блоков. Принимает 0 или 1. По умолчанию - 0 (выключено). +Использование кэша несжатых блоков (только для таблиц семейства MergeTree) может существенно сократить задержку и увеличить пропускную способность при работе с большим количеством коротких запросов. Включите эту настройку для пользователей, от которых идут частые короткие запросы. Также обратите внимание на конфигурационный параметр [uncompressed_cache_size](../server_settings/settings.md#server-settings-uncompressed_cache_size) (настраивается только в конфигурационном файле) - размер кэша разжатых блоков. По умолчанию - 8 GiB. Кэш разжатых блоков заполняется по мере надобности, а наиболее невостребованные данные автоматически удаляются. -Использовать ли кэш разжатых блоков. Принимает 0 или 1. По умолчанию - 1 (включено). -Кэш разжатых блоков (только для таблиц семейства MergeTree) позволяет существенно уменьшить задержки и увеличить пропускную способность при обработке большого количества коротких запросов. Включите эту настройку для пользователей, от которых идут частые короткие запросы. Также обратите внимание на конфигурационный параметр uncompressed_cache_size (настраивается только в конфигурационном файле) - размер кэша разжатых блоков. По умолчанию - 8 GiB. Кэш разжатых блоков заполняется по мере надобности; наиболее невостребованные данные автоматически удаляются. - -Для запросов, читающих хоть немного приличный объём данных (миллион строк и больше), кэш разжатых блоков автоматически выключается, чтобы оставить место для действительно мелких запросов. Поэтому, можно держать настройку use_uncompressed_cache всегда выставленной в 1. +Для запросов, читающих хоть немного приличный объём данных (миллион строк и больше), кэш разжатых блоков автоматически выключается, чтобы оставить место для действительно мелких запросов. Поэтому, можно держать настройку `use_uncompressed_cache` всегда выставленной в 1. ## replace_running_query -При использовании HTTP-интерфейса, может быть передан параметр query_id - произвольная строка, являющаяся идентификатором запроса. +При использовании интерфейса HTTP может быть передан параметр query_id. Это любая строка, которая служит идентификатором запроса. Если в этот момент, уже существует запрос от того же пользователя с тем же query_id, то поведение определяется параметром replace_running_query. `0` - (по умолчанию) кинуть исключение (не давать выполнить запрос, если запрос с таким же query_id уже выполняется); @@ -243,34 +303,35 @@ ClickHouse применяет настройку в тех случаях, ко Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/). Значение параметра зависит от формата. - ## stream_flush_interval_ms -Работает для таблиц со стриммингом в случае тайм-аута, или когда поток генерирует [max_insert_block_size](#settings-max_insert_block_size) строк. +Работает для таблиц со стриммингом в случае тайм-аута, или когда поток генерирует [max_insert_block_size](#settings-max_insert_block_size) строк. Значение по умолчанию - 7500. Чем меньше значение, тем чаще данные сбрасываются в таблицу. Установка слишком низкого значения приводит к снижению производительности. - ## load_balancing На какие реплики (среди живых реплик) предпочитать отправлять запрос (при первой попытке) при распределённой обработке запроса. ### random (по умолчанию) + Для каждой реплики считается количество ошибок. Запрос отправляется на реплику с минимальным числом ошибок, а если таких несколько, то на случайную из них. Недостатки: не учитывается близость серверов; если на репликах оказались разные данные, то вы будете получать так же разные данные. ### nearest_hostname -Для каждой реплики считается количество ошибок. Каждые 5 минут, число ошибок целочисленно делится на 2 - таким образом, обеспечивается расчёт числа ошибок за недавнее время с экспоненциальным сглаживанием. Если есть одна реплика с минимальным числом ошибок (то есть, на других репликах недавно были ошибки) - запрос отправляется на неё. Если есть несколько реплик с одинаковым минимальным числом ошибок, то запрос отправляется на реплику, имя хоста которой в конфигурационном файле минимально отличается от имени хоста сервера (по количеству отличающихся символов на одинаковых позициях, до минимальной длины обеих имён хостов). + +Для каждой реплики считается количество ошибок. Каждые 5 минут, число ошибок целочисленно делится на 2. Таким образом, обеспечивается расчёт числа ошибок за недавнее время с экспоненциальным сглаживанием. Если есть одна реплика с минимальным числом ошибок (то есть, на других репликах недавно были ошибки) - запрос отправляется на неё. Если есть несколько реплик с одинаковым минимальным числом ошибок, то запрос отправляется на реплику, имя хоста которой в конфигурационном файле минимально отличается от имени хоста сервера (по количеству отличающихся символов на одинаковых позициях, до минимальной длины обеих имён хостов). Для примера, example01-01-1 и example01-01-2.yandex.ru отличаются в одной позиции, а example01-01-1 и example01-02-2 - в двух. -Этот способ может показаться несколько дурацким, но он не использует внешние данные о топологии сети, и не сравнивает IP-адреса, что было бы сложным для наших IPv6-адресов. +Этот метод может показаться примитивным, но он не требует внешних данных о топологии сети и не сравнивает IP-адреса, что было бы сложно для наших IPv6-адресов. Таким образом, если есть равнозначные реплики, предпочитается ближайшая по имени. Также можно сделать предположение, что при отправке запроса на один и тот же сервер, в случае отсутствия сбоев, распределённый запрос будет идти тоже на одни и те же серверы. То есть, даже если на репликах расположены разные данные, запрос будет возвращать в основном одинаковые результаты. ### in_order + Реплики перебираются в таком порядке, в каком они указаны. Количество ошибок не имеет значения. Этот способ подходит для тех случаев, когда вы точно знаете, какая реплика предпочтительнее. @@ -300,7 +361,7 @@ ClickHouse применяет настройку в тех случаях, ко ## min_count_to_compile После скольких раз, когда скомпилированный кусок кода мог пригодиться, выполнить его компиляцию. По умолчанию - 3. -В случае, если значение равно нулю, то компиляция выполняется синхронно, и запрос будет ждать окончания процесса компиляции перед продолжением выполнения. Это можно использовать для тестирования, иначе используйте значения, начиная с 1. Как правило, компиляция занимает по времени около 5-10 секунд. +Для тестирования можно установить значение 0: компиляция выполняется синхронно, и запрос ожидает окончания процесса компиляции перед продолжением выполнения. Во всех остальных случаях используйте значения, начинающиеся с 1. Как правило, компиляция занимает по времени около 5-10 секунд. В случае, если значение равно 1 или больше, компиляция выполняется асинхронно, в отдельном потоке. При готовности результата, он сразу же будет использован, в том числе, уже выполняющимися в данный момент запросами. Скомпилированный код требуется для каждого разного сочетания используемых в запросе агрегатных функций и вида ключей в GROUP BY. @@ -308,7 +369,7 @@ ClickHouse применяет настройку в тех случаях, ко ## input_format_skip_unknown_fields -Если значение истинно, то при выполнении INSERT из входных данных пропускаются (не рассматриваются) колонки с неизвестными именами, иначе в данной ситуации будет сгенерировано исключение. +Если значение равно true, то при выполнении INSERT входные данные из столбцов с неизвестными именами будут пропущены. В противном случае эта ситуация создаст исключение. Работает для форматов JSONEachRow и TSKV. ## insert_sample_with_metadata @@ -323,14 +384,12 @@ ClickHouse применяет настройку в тех случаях, ко Символ, интерпретируемый как разделитель в данных формата CSV. По умолчанию — `,`. - ## join_use_nulls Влияет на поведение [JOIN](../../query_language/select.md). При `join_use_nulls=1` `JOIN` ведёт себя как в стандартном SQL, т.е. если при слиянии возникают пустые ячейки, то тип соответствующего поля преобразуется к [Nullable](../../data_types/nullable.md#data_type-nullable), а пустые ячейки заполняются значениями [NULL](../../query_language/syntax.md). - ## insert_quorum {#settings-insert_quorum} Включает кворумную запись. @@ -338,7 +397,7 @@ ClickHouse применяет настройку в тех случаях, ко - Если `insert_quorum < 2`, то кворумная запись выключена. - Если `insert_quorum >= 2`, то кворумная запись включена. -Значение по умолчанию — 0. +Значение по умолчанию: 0. **Кворумная запись** @@ -353,30 +412,29 @@ ClickHouse применяет настройку в тех случаях, ко - Если количество доступных реплик на момент запроса меньше `insert_quorum`. - При попытке записать данные в момент, когда предыдущий блок ещё не вставлен в `insert_quorum` реплик. Эта ситуация может возникнуть, если пользователь вызвал `INSERT` прежде, чем завершился предыдущий с `insert_quorum`. -**См. также параметры** +**См. также параметры:** - [insert_quorum_timeout](#settings-insert_quorum_timeout) - [select_sequential_consistency](#settings-select_sequential_consistency) - ## insert_quorum_timeout {#settings-insert_quorum_timeout} Время ожидания кворумной записи в секундах. Если время прошло, а запись так не состоялась, то ClickHouse сгенерирует исключение и клиент должен повторить запрос на запись того же блока на эту же или любую другую реплику. -По умолчанию 60 секунд. +Значение по умолчанию: 60 секунд. -**См. также параметры** +**См. также параметры:** - [insert_quorum](#settings-insert_quorum) - [select_sequential_consistency](#settings-select_sequential_consistency) - ## select_sequential_consistency {#settings-select_sequential_consistency} Включение/выключение последовательной консистентности для запросов `SELECT`: -- 0 — выключена. Значение по умолчанию. +- 0 — выключена. - 1 — включена. +Значение по умолчанию: 0. Когда последовательная консистентность включена, то ClickHouse позволит клиенту выполнить запрос `SELECT` только к тем репликам, которые содержат данные всех предыдущих запросов `INSERT`, выполненных с `insert_quorum`. Если клиент обратится к неполной реплике, то ClickHouse сгенерирует исключение. В запросе SELECT не будут участвовать данные, которые ещё не были записаны на кворум реплик. diff --git a/docs/toc_ru.yml b/docs/toc_ru.yml index 33e05c23677..c3a7afb8e05 100644 --- a/docs/toc_ru.yml +++ b/docs/toc_ru.yml @@ -160,7 +160,7 @@ nav: - 'Введение': 'operations/index.md' - 'Требования': 'operations/requirements.md' - 'Мониторинг': 'operations/monitoring.md' - - 'Решение проблем': 'operations/troubleshooting.md' + - 'Устранение неисправностей': 'operations/troubleshooting.md' - 'Советы по эксплуатации': 'operations/tips.md' - 'Обновление ClickHouse': 'operations/update.md' - 'Права доступа': 'operations/access_rights.md' From 3218f5003f132a4124746577ff7fb3cea9f4c654 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 12 Feb 2019 15:11:45 +0300 Subject: [PATCH 19/40] fix full/right join different number of columns --- dbms/src/Interpreters/Join.cpp | 3 ++- .../00819_full_join_wrong_columns_in_block.reference | 8 ++++++++ .../00819_full_join_wrong_columns_in_block.sql | 9 +++++++++ 3 files changed, 19 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference create mode 100644 dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 9d5c19fe565..e1a30c5778d 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -1207,7 +1207,8 @@ private: for (size_t i = 0; i < right_sample_block.columns(); ++i) { const ColumnWithTypeAndName & src_column = right_sample_block.getByPosition(i); - result_sample_block.insert(src_column.cloneEmpty()); + if (!result_sample_block.has(src_column.name)) + result_sample_block.insert(src_column.cloneEmpty()); } const auto & key_names_right = parent.key_names_right; diff --git a/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference b/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference new file mode 100644 index 00000000000..074ee47e294 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference @@ -0,0 +1,8 @@ +1 x x +1 x x +1 x x +1 x x +1 x x +1 x x +1 x x +1 x x diff --git a/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql b/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql new file mode 100644 index 00000000000..ba746b62c09 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql @@ -0,0 +1,9 @@ +SELECT * FROM (SELECT 1 AS a, 'x' AS b) join (SELECT 1 as a, 'y' as b) using a; +SELECT * FROM (SELECT 1 AS a, 'x' AS b) left join (SELECT 1 as a, 'y' as b) using a; +SELECT * FROM (SELECT 1 AS a, 'x' AS b) full join (SELECT 1 as a, 'y' as b) using a; +SELECT * FROM (SELECT 1 AS a, 'x' AS b) right join (SELECT 1 as a, 'y' as b) using a; + +SELECT * FROM (SELECT 1 AS a, 'x' AS b) any join (SELECT 1 as a, 'y' as b) using a; +SELECT * FROM (SELECT 1 AS a, 'x' AS b) any left join (SELECT 1 as a, 'y' as b) using a; +SELECT * FROM (SELECT 1 AS a, 'x' AS b) any full join (SELECT 1 as a, 'y' as b) using a; +SELECT * FROM (SELECT 1 AS a, 'x' AS b) any right join (SELECT 1 as a, 'y' as b) using a; From 8f1910729aea22a4b5c64eab64f9f514f28e030b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Feb 2019 15:33:24 +0300 Subject: [PATCH 20/40] Fixed ubsan build #4322 --- dbms/src/Functions/arrayWithConstant.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dbms/src/Functions/arrayWithConstant.cpp b/dbms/src/Functions/arrayWithConstant.cpp index 643309760d6..d9a2668c1e1 100644 --- a/dbms/src/Functions/arrayWithConstant.cpp +++ b/dbms/src/Functions/arrayWithConstant.cpp @@ -59,9 +59,16 @@ public: ColumnArray::Offset offset = 0; for (size_t i = 0; i < num_rows; ++i) { + auto array_size = col_num->getInt(i); + + if (unlikely(array_size) < 0) + throw Exception("Array size cannot be negative: while executing function " + getName(), ErrorCodes::TOO_LARGE_ARRAY_SIZE); + offset += col_num->getUInt(i); + if (unlikely(offset > max_arrays_size_in_block)) throw Exception("Too large array size while executing function " + getName(), ErrorCodes::TOO_LARGE_ARRAY_SIZE); + offsets.push_back(offset); } From bdefba6c329b58caa014c0fb8de7e44ebe3ccb72 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 12 Feb 2019 15:59:40 +0300 Subject: [PATCH 21/40] fix link order --- dbms/src/Storages/Kafka/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/Kafka/CMakeLists.txt b/dbms/src/Storages/Kafka/CMakeLists.txt index c764a1b7fd5..520f1311d1d 100644 --- a/dbms/src/Storages/Kafka/CMakeLists.txt +++ b/dbms/src/Storages/Kafka/CMakeLists.txt @@ -2,7 +2,7 @@ if(USE_RDKAFKA) include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(clickhouse_storage_kafka .) add_library(clickhouse_storage_kafka ${LINK_MODE} ${clickhouse_storage_kafka_sources}) - target_link_libraries(clickhouse_storage_kafka PRIVATE clickhouse_common_io ${RDKAFKA_LIBRARY} ${CPPKAFKA_LIBRARY}) + target_link_libraries(clickhouse_storage_kafka PRIVATE clickhouse_common_io ${CPPKAFKA_LIBRARY} ${RDKAFKA_LIBRARY}) if(NOT USE_INTERNAL_RDKAFKA_LIBRARY) target_include_directories(clickhouse_storage_kafka SYSTEM BEFORE PRIVATE ${RDKAFKA_INCLUDE_DIR}) endif() From 2aafcb8401ffd78f98201b536d5f1653b9ec4df6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Feb 2019 16:51:34 +0300 Subject: [PATCH 22/40] Fixed ubsan build #4322 --- dbms/src/Functions/arrayWithConstant.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/arrayWithConstant.cpp b/dbms/src/Functions/arrayWithConstant.cpp index d9a2668c1e1..f9d1aff5f43 100644 --- a/dbms/src/Functions/arrayWithConstant.cpp +++ b/dbms/src/Functions/arrayWithConstant.cpp @@ -64,7 +64,7 @@ public: if (unlikely(array_size) < 0) throw Exception("Array size cannot be negative: while executing function " + getName(), ErrorCodes::TOO_LARGE_ARRAY_SIZE); - offset += col_num->getUInt(i); + offset += array_size; if (unlikely(offset > max_arrays_size_in_block)) throw Exception("Too large array size while executing function " + getName(), ErrorCodes::TOO_LARGE_ARRAY_SIZE); From edefa194201735e14ed26980c1e5fcb7a23c0b31 Mon Sep 17 00:00:00 2001 From: proller Date: Tue, 12 Feb 2019 17:08:05 +0300 Subject: [PATCH 23/40] Move odbc-brigde to clickhouse-common-static --- debian/clickhouse-common-static.install | 1 + debian/clickhouse-server.install | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/debian/clickhouse-common-static.install b/debian/clickhouse-common-static.install index a2f5b69c964..6666b090272 100644 --- a/debian/clickhouse-common-static.install +++ b/debian/clickhouse-common-static.install @@ -1,3 +1,4 @@ usr/bin/clickhouse +usr/bin/clickhouse-odbc-bridge etc/security/limits.d/clickhouse.conf usr/share/clickhouse/* diff --git a/debian/clickhouse-server.install b/debian/clickhouse-server.install index bc3902ed41a..f69969a6084 100644 --- a/debian/clickhouse-server.install +++ b/debian/clickhouse-server.install @@ -2,7 +2,6 @@ usr/bin/clickhouse-server usr/bin/clickhouse-clang usr/bin/clickhouse-lld usr/bin/clickhouse-copier -usr/bin/clickhouse-odbc-bridge usr/bin/clickhouse-report etc/clickhouse-server/config.xml etc/clickhouse-server/users.xml From 83d461975e4db37a5e4c4b2ddc28121dfc674dd3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 12 Feb 2019 17:18:00 +0300 Subject: [PATCH 24/40] Function toStartOfDay() now can receive a date. --- dbms/src/Functions/DateTimeTransforms.h | 4 +- .../FunctionDateOrDateTimeToSomething.h | 28 +++++--- dbms/src/Functions/toStartOfInterval.cpp | 34 ++++++--- .../0_stateless/00189_time_zones.reference | 59 ++++++++++++++++ .../queries/0_stateless/00189_time_zones.sql | 69 ++++++++++++++++++- 5 files changed, 171 insertions(+), 23 deletions(-) diff --git a/dbms/src/Functions/DateTimeTransforms.h b/dbms/src/Functions/DateTimeTransforms.h index 5e3b540533c..6890b513602 100644 --- a/dbms/src/Functions/DateTimeTransforms.h +++ b/dbms/src/Functions/DateTimeTransforms.h @@ -65,9 +65,9 @@ struct ToStartOfDayImpl { return time_zone.toDate(t); } - static inline UInt32 execute(UInt16, const DateLUTImpl &) + static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { - return dateIsNotSupported(name); + return time_zone.toDate(DayNum(d)); } using FactorTransform = ZeroTransform; diff --git a/dbms/src/Functions/FunctionDateOrDateTimeToSomething.h b/dbms/src/Functions/FunctionDateOrDateTimeToSomething.h index 19baf8569a2..bb32230a5b1 100644 --- a/dbms/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/dbms/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -37,23 +37,33 @@ public: if (arguments.size() == 1) { if (!isDateOrDateTime(arguments[0].type)) - throw Exception("Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + - ". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + + ". Should be a date or a date with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } else if (arguments.size() == 2) { - if (!WhichDataType(arguments[0].type).isDateTime() - || !WhichDataType(arguments[1].type).isString()) + if (!isDateOrDateTime(arguments[0].type)) + throw Exception( + "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + + ". Should be a date or a date with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!isString(arguments[1].type)) throw Exception( "Function " + getName() + " supports 1 or 2 arguments. The 1st argument " - "must be of type Date or DateTime. The 2nd argument (optional) must be " - "a constant string with timezone name. The timezone argument is allowed " - "only when the 1st argument has the type DateTime", + "must be of type Date or DateTime. The 2nd argument (optional) must be " + "a constant string with timezone name", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (isDate(arguments[0].type) && std::is_same_v) + throw Exception( + "The timezone argument of function " + getName() + " is allowed only when the 1st argument has the type DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } else - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 1 or 2", + throw Exception( + "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + + ", should be 1 or 2", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); /// For DateTime, if time zone is specified, attach it to type. diff --git a/dbms/src/Functions/toStartOfInterval.cpp b/dbms/src/Functions/toStartOfInterval.cpp index bae34568811..21e500602e1 100644 --- a/dbms/src/Functions/toStartOfInterval.cpp +++ b/dbms/src/Functions/toStartOfInterval.cpp @@ -142,41 +142,54 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - auto check_date_time_argument = [&] { + bool first_argument_is_date = false; + auto check_first_argument = [&] + { if (!isDateOrDateTime(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() + ". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + first_argument_is_date = isDate(arguments[0].type); }; const DataTypeInterval * interval_type = nullptr; - auto check_interval_argument = [&] { + bool result_type_is_date = false; + auto check_interval_argument = [&] + { interval_type = checkAndGetDataType(arguments[1].type.get()); if (!interval_type) throw Exception( "Illegal type " + arguments[1].type->getName() + " of argument of function " + getName() + ". Should be an interval of time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + result_type_is_date = (interval_type->getKind() == DataTypeInterval::Year) + || (interval_type->getKind() == DataTypeInterval::Quarter) || (interval_type->getKind() == DataTypeInterval::Month) + || (interval_type->getKind() == DataTypeInterval::Week); }; - auto check_timezone_argument = [&] { + auto check_timezone_argument = [&] + { if (!WhichDataType(arguments[2].type).isString()) throw Exception( "Illegal type " + arguments[2].type->getName() + " of argument of function " + getName() - + ". This argument is optional and must be a constant string with timezone name" - ". This argument is allowed only when the 1st argument has the type DateTime", + + ". This argument is optional and must be a constant string with timezone name", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (first_argument_is_date && result_type_is_date) + throw Exception( + "The timezone argument of function " + getName() + " with interval type " + interval_type->kindToString() + + " is allowed only when the 1st argument has the type DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); }; if (arguments.size() == 2) { - check_date_time_argument(); + check_first_argument(); check_interval_argument(); } else if (arguments.size() == 3) { - check_date_time_argument(); + check_first_argument(); check_interval_argument(); check_timezone_argument(); } @@ -188,11 +201,10 @@ public: ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - if ((interval_type->getKind() == DataTypeInterval::Second) || (interval_type->getKind() == DataTypeInterval::Minute) - || (interval_type->getKind() == DataTypeInterval::Hour) || (interval_type->getKind() == DataTypeInterval::Day)) - return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); - else + if (result_type_is_date) return std::make_shared(); + else + return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); } bool useDefaultImplementationForConstants() const override { return true; } diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.reference b/dbms/tests/queries/0_stateless/00189_time_zones.reference index 5de17b8bb37..8e0abb19516 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.reference +++ b/dbms/tests/queries/0_stateless/00189_time_zones.reference @@ -1,88 +1,137 @@ +toStartOfDay +2014-09-30 00:00:00 +2014-09-30 00:00:00 +2014-09-30 00:00:00 +2014-10-01 00:00:00 +2014-09-30 00:00:00 +2014-09-30 00:00:00 +2014-09-30 00:00:00 +2014-09-30 00:00:00 +2014-09-30 00:00:00 +2014-09-30 00:00:00 +toMonday 2014-12-29 2014-12-22 2014-12-22 2014-12-29 2014-12-22 +2014-12-29 +2014-12-29 +2014-12-29 +2014-12-29 +2014-12-29 +toStartOfMonth 2014-12-01 2014-12-01 2014-12-01 2014-12-01 2014-12-01 +2014-12-01 +2014-12-01 +2014-12-01 +2014-12-01 +2014-12-01 +toStartOfQuarter 2014-07-01 2014-07-01 2014-07-01 2014-10-01 2014-07-01 +2014-07-01 +2014-07-01 +2014-07-01 +2014-07-01 +2014-07-01 +toStartOfYear 2014-01-01 2014-01-01 2014-01-01 2014-01-01 2014-01-01 +2014-01-01 +2014-01-01 +2014-01-01 +2014-01-01 +2014-01-01 +toTime 1970-01-02 12:00:00 1970-01-02 12:00:00 1970-01-02 10:00:00 1970-01-02 11:00:00 1970-01-02 09:00:00 1970-01-02 10:00:00 1970-01-02 18:00:00 1970-01-02 18:00:00 1970-01-02 01:00:00 1970-01-02 01:00:00 +toYear 2014 2014 2014 2014 2014 +toMonth 9 9 9 10 9 +toDayOfMonth 30 30 30 1 30 +toDayOfWeek 2 2 2 3 2 +toHour 23 21 20 4 11 +toMinute 50 50 50 50 50 +toSecond 0 0 0 0 0 +toStartOfMinute 2019-02-06 22:57:00 2019-02-06 20:57:00 2019-02-06 19:57:00 2019-02-07 04:57:00 2019-02-06 11:57:00 +toStartOfFiveMinute 2019-02-06 22:55:00 2019-02-06 20:55:00 2019-02-06 19:55:00 2019-02-07 04:55:00 2019-02-06 11:55:00 +toStartOfTenMinutes 2019-02-06 22:50:00 2019-02-06 20:50:00 2019-02-06 19:50:00 2019-02-07 04:50:00 2019-02-06 11:50:00 +toStartOfFifteenMinutes 2019-02-06 22:45:00 2019-02-06 20:45:00 2019-02-06 19:45:00 2019-02-07 04:45:00 2019-02-06 11:45:00 +toStartOfHour 2019-02-06 22:00:00 2019-02-06 20:00:00 2019-02-06 19:00:00 2019-02-07 04:00:00 2019-02-06 11:00:00 +toStartOfInterval 2019-01-01 2018-01-01 2015-01-01 @@ -125,40 +174,48 @@ 2019-02-06 00:00:00 2019-02-05 00:00:00 2019-02-03 00:00:00 +toRelativeYearNum 44 44 44 44 44 +toRelativeMonthNum 536 536 536 537 536 +toRelativeWeekNum 2335 2335 2335 2335 2335 +toRelativeDayNum 16343 16343 16343 16344 16343 +toRelativeHourNum 392251 392251 392251 392251 +toRelativeMinuteNum 23535110 23535110 23535110 23535110 23535110 +toRelativeSecondNum 1412106600 1412106600 1412106600 1412106600 1412106600 +toDate 2014-09-30 2014-09-30 2014-09-30 @@ -169,11 +226,13 @@ 2014-09-30 2014-10-01 2014-09-30 +toString 2015-07-15 13:30:00 2015-07-15 12:30:00 2015-07-15 11:30:00 2015-07-15 19:30:00 2015-07-15 02:30:00 +toUnixTimestamp 1426415400 1426422600 1426426200 diff --git a/dbms/tests/queries/0_stateless/00189_time_zones.sql b/dbms/tests/queries/0_stateless/00189_time_zones.sql index 41936a5c145..45d8dc8e7fc 100644 --- a/dbms/tests/queries/0_stateless/00189_time_zones.sql +++ b/dbms/tests/queries/0_stateless/00189_time_zones.sql @@ -5,41 +5,84 @@ /* timestamp 1428310800 == 2015-04-06 12:00:00 (Europe/Moscow) */ /* timestamp 1436956200 == 2015-07-15 13:30:00 (Europe/Moscow) */ /* timestamp 1426415400 == 2015-03-15 13:30:00 (Europe/Moscow) */ +/* timestamp 1549483055 == 2019-02-06 22:57:35 (Europe/Moscow) */ +/* date 16343 == 2014-09-30 */ +/* date 16433 == 2014-12-29 */ +/* date 17933 == 2019-02-06 */ + +/* toStartOfDay */ + +SELECT 'toStartOfDay'; +SELECT toStartOfDay(toDateTime(1412106600), 'Europe/Moscow'); +SELECT toStartOfDay(toDateTime(1412106600), 'Europe/Paris'); +SELECT toStartOfDay(toDateTime(1412106600), 'Europe/London'); +SELECT toStartOfDay(toDateTime(1412106600), 'Asia/Tokyo'); +SELECT toStartOfDay(toDateTime(1412106600), 'Pacific/Pitcairn'); +SELECT toStartOfDay(toDate(16343), 'Europe/Moscow'); +SELECT toStartOfDay(toDate(16343), 'Europe/Paris'); +SELECT toStartOfDay(toDate(16343), 'Europe/London'); +SELECT toStartOfDay(toDate(16343), 'Asia/Tokyo'); +SELECT toStartOfDay(toDate(16343), 'Pacific/Pitcairn'); /* toMonday */ +SELECT 'toMonday'; SELECT toMonday(toDateTime(1419800400), 'Europe/Moscow'); SELECT toMonday(toDateTime(1419800400), 'Europe/Paris'); SELECT toMonday(toDateTime(1419800400), 'Europe/London'); SELECT toMonday(toDateTime(1419800400), 'Asia/Tokyo'); SELECT toMonday(toDateTime(1419800400), 'Pacific/Pitcairn'); +SELECT toMonday(toDate(16433)); +SELECT toMonday(toDate(16433)); +SELECT toMonday(toDate(16433)); +SELECT toMonday(toDate(16433)); +SELECT toMonday(toDate(16433)); /* toStartOfMonth */ +SELECT 'toStartOfMonth'; SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Moscow'); SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/Paris'); SELECT toStartOfMonth(toDateTime(1419800400), 'Europe/London'); SELECT toStartOfMonth(toDateTime(1419800400), 'Asia/Tokyo'); SELECT toStartOfMonth(toDateTime(1419800400), 'Pacific/Pitcairn'); +SELECT toStartOfMonth(toDate(16433)); +SELECT toStartOfMonth(toDate(16433)); +SELECT toStartOfMonth(toDate(16433)); +SELECT toStartOfMonth(toDate(16433)); +SELECT toStartOfMonth(toDate(16433)); /* toStartOfQuarter */ +SELECT 'toStartOfQuarter'; SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Moscow'); SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/Paris'); SELECT toStartOfQuarter(toDateTime(1412106600), 'Europe/London'); SELECT toStartOfQuarter(toDateTime(1412106600), 'Asia/Tokyo'); SELECT toStartOfQuarter(toDateTime(1412106600), 'Pacific/Pitcairn'); +SELECT toStartOfQuarter(toDate(16343)); +SELECT toStartOfQuarter(toDate(16343)); +SELECT toStartOfQuarter(toDate(16343)); +SELECT toStartOfQuarter(toDate(16343)); +SELECT toStartOfQuarter(toDate(16343)); /* toStartOfYear */ +SELECT 'toStartOfYear'; SELECT toStartOfYear(toDateTime(1419800400), 'Europe/Moscow'); SELECT toStartOfYear(toDateTime(1419800400), 'Europe/Paris'); SELECT toStartOfYear(toDateTime(1419800400), 'Europe/London'); SELECT toStartOfYear(toDateTime(1419800400), 'Asia/Tokyo'); SELECT toStartOfYear(toDateTime(1419800400), 'Pacific/Pitcairn'); +SELECT toStartOfYear(toDate(16433)); +SELECT toStartOfYear(toDate(16433)); +SELECT toStartOfYear(toDate(16433)); +SELECT toStartOfYear(toDate(16433)); +SELECT toStartOfYear(toDate(16433)); /* toTime */ +SELECT 'toTime'; SELECT toString(toTime(toDateTime(1420102800), 'Europe/Moscow'), 'Europe/Moscow'), toString(toTime(toDateTime(1428310800), 'Europe/Moscow'), 'Europe/Moscow'); SELECT toString(toTime(toDateTime(1420102800), 'Europe/Paris'), 'Europe/Paris'), toString(toTime(toDateTime(1428310800), 'Europe/Paris'), 'Europe/Paris'); SELECT toString(toTime(toDateTime(1420102800), 'Europe/London'), 'Europe/London'), toString(toTime(toDateTime(1428310800), 'Europe/London'), 'Europe/London'); @@ -48,6 +91,7 @@ SELECT toString(toTime(toDateTime(1420102800), 'Pacific/Pitcairn'), 'Pacific/Pit /* toYear */ +SELECT 'toYear'; SELECT toYear(toDateTime(1412106600), 'Europe/Moscow'); SELECT toYear(toDateTime(1412106600), 'Europe/Paris'); SELECT toYear(toDateTime(1412106600), 'Europe/London'); @@ -56,6 +100,7 @@ SELECT toYear(toDateTime(1412106600), 'Pacific/Pitcairn'); /* toMonth */ +SELECT 'toMonth'; SELECT toMonth(toDateTime(1412106600), 'Europe/Moscow'); SELECT toMonth(toDateTime(1412106600), 'Europe/Paris'); SELECT toMonth(toDateTime(1412106600), 'Europe/London'); @@ -64,6 +109,7 @@ SELECT toMonth(toDateTime(1412106600), 'Pacific/Pitcairn'); /* toDayOfMonth */ +SELECT 'toDayOfMonth'; SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/Moscow'); SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/Paris'); SELECT toDayOfMonth(toDateTime(1412106600), 'Europe/London'); @@ -72,6 +118,7 @@ SELECT toDayOfMonth(toDateTime(1412106600), 'Pacific/Pitcairn'); /* toDayOfWeek */ +SELECT 'toDayOfWeek'; SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/Moscow'); SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/Paris'); SELECT toDayOfWeek(toDateTime(1412106600), 'Europe/London'); @@ -80,6 +127,7 @@ SELECT toDayOfWeek(toDateTime(1412106600), 'Pacific/Pitcairn'); /* toHour */ +SELECT 'toHour'; SELECT toHour(toDateTime(1412106600), 'Europe/Moscow'); SELECT toHour(toDateTime(1412106600), 'Europe/Paris'); SELECT toHour(toDateTime(1412106600), 'Europe/London'); @@ -88,6 +136,7 @@ SELECT toHour(toDateTime(1412106600), 'Pacific/Pitcairn'); /* toMinute */ +SELECT 'toMinute'; SELECT toMinute(toDateTime(1412106600), 'Europe/Moscow'); SELECT toMinute(toDateTime(1412106600), 'Europe/Paris'); SELECT toMinute(toDateTime(1412106600), 'Europe/London'); @@ -96,6 +145,7 @@ SELECT toMinute(toDateTime(1412106600), 'Pacific/Pitcairn'); /* toSecond */ +SELECT 'toSecond'; SELECT toSecond(toDateTime(1412106600), 'Europe/Moscow'); SELECT toSecond(toDateTime(1412106600), 'Europe/Paris'); SELECT toSecond(toDateTime(1412106600), 'Europe/London'); @@ -104,6 +154,7 @@ SELECT toSecond(toDateTime(1412106600), 'Pacific/Pitcairn'); /* toStartOfMinute */ +SELECT 'toStartOfMinute'; SELECT toString(toStartOfMinute(toDateTime(1549483055), 'Europe/Moscow'), 'Europe/Moscow'); SELECT toString(toStartOfMinute(toDateTime(1549483055), 'Europe/Paris'), 'Europe/Paris'); SELECT toString(toStartOfMinute(toDateTime(1549483055), 'Europe/London'), 'Europe/London'); @@ -112,6 +163,7 @@ SELECT toString(toStartOfMinute(toDateTime(1549483055), 'Pacific/Pitcairn'), 'Pa /* toStartOfFiveMinute */ +SELECT 'toStartOfFiveMinute'; SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Europe/Moscow'), 'Europe/Moscow'); SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Europe/Paris'), 'Europe/Paris'); SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Europe/London'), 'Europe/London'); @@ -120,14 +172,16 @@ SELECT toString(toStartOfFiveMinute(toDateTime(1549483055), 'Pacific/Pitcairn'), /* toStartOfTenMinutes */ +SELECT 'toStartOfTenMinutes'; SELECT toString(toStartOfTenMinutes(toDateTime(1549483055), 'Europe/Moscow'), 'Europe/Moscow'); SELECT toString(toStartOfTenMinutes(toDateTime(1549483055), 'Europe/Paris'), 'Europe/Paris'); SELECT toString(toStartOfTenMinutes(toDateTime(1549483055), 'Europe/London'), 'Europe/London'); SELECT toString(toStartOfTenMinutes(toDateTime(1549483055), 'Asia/Tokyo'), 'Asia/Tokyo'); SELECT toString(toStartOfTenMinutes(toDateTime(1549483055), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); -/* toStartOfTenMinutes */ +/* toStartOfFifteenMinutes */ +SELECT 'toStartOfFifteenMinutes'; SELECT toString(toStartOfFifteenMinutes(toDateTime(1549483055), 'Europe/Moscow'), 'Europe/Moscow'); SELECT toString(toStartOfFifteenMinutes(toDateTime(1549483055), 'Europe/Paris'), 'Europe/Paris'); SELECT toString(toStartOfFifteenMinutes(toDateTime(1549483055), 'Europe/London'), 'Europe/London'); @@ -136,6 +190,7 @@ SELECT toString(toStartOfFifteenMinutes(toDateTime(1549483055), 'Pacific/Pitcair /* toStartOfHour */ +SELECT 'toStartOfHour'; SELECT toString(toStartOfHour(toDateTime(1549483055), 'Europe/Moscow'), 'Europe/Moscow'); SELECT toString(toStartOfHour(toDateTime(1549483055), 'Europe/Paris'), 'Europe/Paris'); SELECT toString(toStartOfHour(toDateTime(1549483055), 'Europe/London'), 'Europe/London'); @@ -143,6 +198,8 @@ SELECT toString(toStartOfHour(toDateTime(1549483055), 'Asia/Tokyo'), 'Asia/Tokyo SELECT toString(toStartOfHour(toDateTime(1549483055), 'Pacific/Pitcairn'), 'Pacific/Pitcairn'); /* toStartOfInterval */ + +SELECT 'toStartOfInterval'; SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 1 year, 'Europe/Moscow'); SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 2 year, 'Europe/Moscow'); SELECT toStartOfInterval(toDateTime(1549483055), INTERVAL 5 year, 'Europe/Moscow'); @@ -188,6 +245,7 @@ SELECT toString(toStartOfInterval(toDate(17933), INTERVAL 5 day, 'Europe/Moscow' /* toRelativeYearNum */ +SELECT 'toRelativeYearNum'; SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/Moscow') - toRelativeYearNum(toDateTime(0), 'Europe/Moscow'); SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/Paris') - toRelativeYearNum(toDateTime(0), 'Europe/Paris'); SELECT toRelativeYearNum(toDateTime(1412106600), 'Europe/London') - toRelativeYearNum(toDateTime(0), 'Europe/London'); @@ -196,6 +254,7 @@ SELECT toRelativeYearNum(toDateTime(1412106600), 'Pacific/Pitcairn') - toRelativ /* toRelativeMonthNum */ +SELECT 'toRelativeMonthNum'; SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/Moscow') - toRelativeMonthNum(toDateTime(0), 'Europe/Moscow'); SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/Paris') - toRelativeMonthNum(toDateTime(0), 'Europe/Paris'); SELECT toRelativeMonthNum(toDateTime(1412106600), 'Europe/London') - toRelativeMonthNum(toDateTime(0), 'Europe/London'); @@ -204,6 +263,7 @@ SELECT toRelativeMonthNum(toDateTime(1412106600), 'Pacific/Pitcairn') - toRelati /* toRelativeWeekNum */ +SELECT 'toRelativeWeekNum'; SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/Moscow') - toRelativeWeekNum(toDateTime(0), 'Europe/Moscow'); SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/Paris') - toRelativeWeekNum(toDateTime(0), 'Europe/Paris'); SELECT toRelativeWeekNum(toDateTime(1412106600), 'Europe/London') - toRelativeWeekNum(toDateTime(0), 'Europe/London'); @@ -212,6 +272,7 @@ SELECT toRelativeWeekNum(toDateTime(1412106600), 'Pacific/Pitcairn') - toRelativ /* toRelativeDayNum */ +SELECT 'toRelativeDayNum'; SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/Moscow') - toRelativeDayNum(toDateTime(0), 'Europe/Moscow'); SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/Paris') - toRelativeDayNum(toDateTime(0), 'Europe/Paris'); SELECT toRelativeDayNum(toDateTime(1412106600), 'Europe/London') - toRelativeDayNum(toDateTime(0), 'Europe/London'); @@ -220,6 +281,7 @@ SELECT toRelativeDayNum(toDateTime(1412106600), 'Pacific/Pitcairn') - toRelative /* toRelativeHourNum */ +SELECT 'toRelativeHourNum'; SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/Moscow') - toRelativeHourNum(toDateTime(0), 'Europe/Moscow'); SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/Paris') - toRelativeHourNum(toDateTime(0), 'Europe/Paris'); SELECT toRelativeHourNum(toDateTime(1412106600), 'Europe/London') - toRelativeHourNum(toDateTime(0), 'Europe/London'); @@ -228,6 +290,7 @@ SELECT toRelativeHourNum(toDateTime(1412106600), 'Asia/Tokyo') - toRelativeHourN /* toRelativeMinuteNum */ +SELECT 'toRelativeMinuteNum'; SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/Moscow') - toRelativeMinuteNum(toDateTime(0), 'Europe/Moscow'); SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/Paris') - toRelativeMinuteNum(toDateTime(0), 'Europe/Paris'); SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Europe/London') - toRelativeMinuteNum(toDateTime(0), 'Europe/London'); @@ -236,6 +299,7 @@ SELECT toRelativeMinuteNum(toDateTime(1412106600), 'Pacific/Pitcairn') - toRelat /* toRelativeSecondNum */ +SELECT 'toRelativeSecondNum'; SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/Moscow') - toRelativeSecondNum(toDateTime(0), 'Europe/Moscow'); SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/Paris') - toRelativeSecondNum(toDateTime(0), 'Europe/Paris'); SELECT toRelativeSecondNum(toDateTime(1412106600), 'Europe/London') - toRelativeSecondNum(toDateTime(0), 'Europe/London'); @@ -244,6 +308,7 @@ SELECT toRelativeSecondNum(toDateTime(1412106600), 'Pacific/Pitcairn') - toRelat /* toDate */ +SELECT 'toDate'; SELECT toDate(toDateTime(1412106600), 'Europe/Moscow'); SELECT toDate(toDateTime(1412106600), 'Europe/Paris'); SELECT toDate(toDateTime(1412106600), 'Europe/London'); @@ -258,6 +323,7 @@ SELECT toDate(1412106600, 'Pacific/Pitcairn'); /* toString */ +SELECT 'toString'; SELECT toString(toDateTime(1436956200), 'Europe/Moscow'); SELECT toString(toDateTime(1436956200), 'Europe/Paris'); SELECT toString(toDateTime(1436956200), 'Europe/London'); @@ -266,6 +332,7 @@ SELECT toString(toDateTime(1436956200), 'Pacific/Pitcairn'); /* toUnixTimestamp */ +SELECT 'toUnixTimestamp'; SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Moscow'), 'Europe/Moscow'); SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Moscow'), 'Europe/Paris'); SELECT toUnixTimestamp(toString(toDateTime(1426415400), 'Europe/Moscow'), 'Europe/London'); From a73f29ca2e8941adc7acf8b6efbb06d03f7387ad Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Feb 2019 17:38:29 +0300 Subject: [PATCH 25/40] Fix LowCardinality cache. --- dbms/src/Common/ColumnsHashing.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/src/Common/ColumnsHashing.h b/dbms/src/Common/ColumnsHashing.h index 4f6708ae2f1..436ed43660e 100644 --- a/dbms/src/Common/ColumnsHashing.h +++ b/dbms/src/Common/ColumnsHashing.h @@ -364,7 +364,10 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod } if constexpr (has_mapped) + { + mapped_cache[row] = it->second; return EmplaceResult(it->second, mapped_cache[row], inserted); + } else return EmplaceResult(inserted); } From b95f5196c0f9fab7d4e076f41000c6610396e19f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Feb 2019 17:43:52 +0300 Subject: [PATCH 26/40] Added test. --- .../queries/0_stateless/906_low_cardinality_cache.reference | 1 + dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql | 5 +++++ 2 files changed, 6 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/906_low_cardinality_cache.reference create mode 100644 dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql diff --git a/dbms/tests/queries/0_stateless/906_low_cardinality_cache.reference b/dbms/tests/queries/0_stateless/906_low_cardinality_cache.reference new file mode 100644 index 00000000000..5e2be7a023a --- /dev/null +++ b/dbms/tests/queries/0_stateless/906_low_cardinality_cache.reference @@ -0,0 +1 @@ +100000000 0123456789 diff --git a/dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql b/dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql new file mode 100644 index 00000000000..cafc559d77f --- /dev/null +++ b/dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql @@ -0,0 +1,5 @@ +drop table if exists test.lc; +create table test.lc (b LowCardinality(String)) engine=MergeTree order by b; +insert into test.lc select '0123456789' from numbers(100000000); +select count(), b from test.lc group by b; + From 93e0621a2ca64324c8c8262cf8726e36a473c755 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 12 Feb 2019 18:08:21 +0300 Subject: [PATCH 27/40] fix wrong query in push down test --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 5 +++-- dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp | 1 - dbms/src/Parsers/ASTIdentifier.cpp | 9 +++++++++ dbms/src/Parsers/ASTIdentifier.h | 6 +----- .../queries/0_stateless/00597_push_down_predicate.sql | 4 ++-- 5 files changed, 15 insertions(+), 10 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 2c333040360..62f7f91f197 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -1019,9 +1019,10 @@ void ExpressionAnalyzer::collectUsedColumns() for (NamesAndTypesList::iterator it = source_columns.begin(); it != source_columns.end();) { - unknown_required_source_columns.erase(it->name); + const String & column_name = it->name; + unknown_required_source_columns.erase(column_name); - if (!required.count(it->name)) + if (!required.count(column_name)) source_columns.erase(it++); else ++it; diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 930295a3b5a..85f8f09b867 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -250,7 +250,6 @@ void PredicateExpressionsOptimizer::setNewAliasesForInnerPredicate( name = ast->getAliasOrColumnName(); } - IdentifierSemantic::setNeedLongName(*identifier, false); identifier->setShortName(name); } } diff --git a/dbms/src/Parsers/ASTIdentifier.cpp b/dbms/src/Parsers/ASTIdentifier.cpp index e5500a89bd0..b57db5bef87 100644 --- a/dbms/src/Parsers/ASTIdentifier.cpp +++ b/dbms/src/Parsers/ASTIdentifier.cpp @@ -22,6 +22,15 @@ ASTIdentifier::ASTIdentifier(const String & name_, std::vector && name_p { } +void ASTIdentifier::setShortName(const String & new_name) +{ + name = new_name; + name_parts.clear(); + + semantic->need_long_name = false; + semantic->can_be_alias = true; +} + void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, FormatState &, FormatStateStacked) const { auto format_element = [&](const String & elem_name) diff --git a/dbms/src/Parsers/ASTIdentifier.h b/dbms/src/Parsers/ASTIdentifier.h index 9457b7d9156..b875b7dc91a 100644 --- a/dbms/src/Parsers/ASTIdentifier.h +++ b/dbms/src/Parsers/ASTIdentifier.h @@ -36,11 +36,7 @@ public: bool compound() const { return !name_parts.empty(); } bool isShort() const { return name_parts.empty() || name == name_parts.back(); } - void setShortName(const String & new_name) - { - name = new_name; - name_parts.clear(); - } + void setShortName(const String & new_name); const String & shortName() const { diff --git a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql index 0180fcdeb1e..495c367e3bd 100644 --- a/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql +++ b/dbms/tests/queries/0_stateless/00597_push_down_predicate.sql @@ -31,7 +31,7 @@ SELECT * FROM (SELECT 1 AS id, (SELECT 1) as subquery) WHERE subquery = 1; SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test.test) WHERE a = 3; SELECT * FROM (SELECT toUInt64(b), sum(id) AS b FROM test.test) WHERE `toUInt64(sum(id))` = 3; SELECT date, id, name, value FROM (SELECT date, name, value, min(id) AS id FROM test.test GROUP BY date, name, value) WHERE id = 1; -SELECT * FROM (SELECT toUInt64(table_alias.b) AS a, sum(id) AS b FROM test.test AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3; +SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test.test AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3; SELECT '-------Force push down-------'; SET force_primary_key = 1; @@ -72,7 +72,7 @@ SELECT '-------Push to having expression, need check.-------'; SELECT id FROM (SELECT min(id) AS id FROM test.test) WHERE id = 1; -- { serverError 277 } SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test.test) WHERE a = 3; -- { serverError 277 } SELECT * FROM (SELECT toUInt64(b), sum(id) AS b FROM test.test) WHERE `toUInt64(sum(id))` = 3; -- { serverError 277 } -SELECT * FROM (SELECT toUInt64(table_alias.b) AS a, sum(id) AS b FROM test.test AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3; -- { serverError 277 } +SELECT * FROM (SELECT toUInt64(b) AS a, sum(id) AS b FROM test.test AS table_alias) AS outer_table_alias WHERE outer_table_alias.b = 3; -- { serverError 277 } SELECT '-------Compatibility test-------'; SELECT * FROM (SELECT toInt8(1) AS id, toDate('2000-01-01') AS date FROM system.numbers LIMIT 1) ANY LEFT JOIN (SELECT * FROM test.test) AS b USING date, id WHERE b.date = toDate('2000-01-01'); From 1af4d9b89c0078576ae73147ab6b00d5bda43da7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Feb 2019 18:42:33 +0300 Subject: [PATCH 28/40] Fix type for constant LowCardinality arguments. --- dbms/src/Functions/IFunction.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index a3d34a830a9..2905893458b 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -354,7 +354,11 @@ static ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( { ColumnWithTypeAndName & column = block.getByPosition(arg); if (auto * column_const = checkAndGetColumn(column.column.get())) + { column.column = column_const->removeLowCardinality()->cloneResized(num_rows); + if (auto * low_cardinality_type = checkAndGetDataType(column.type.get())) + column.type = removeLowCardinality() ///low_cardinality_type->getDictionaryType(); + } else if (auto * low_cardinality_column = checkAndGetColumn(column.column.get())) { auto * low_cardinality_type = checkAndGetDataType(column.type.get()); From bb686b3154920202841dc2de934c9891973f539e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Feb 2019 18:43:58 +0300 Subject: [PATCH 29/40] Fix type for constant LowCardinality arguments. --- dbms/src/Functions/IFunction.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 2905893458b..069a96f9ad8 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -356,8 +356,7 @@ static ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( if (auto * column_const = checkAndGetColumn(column.column.get())) { column.column = column_const->removeLowCardinality()->cloneResized(num_rows); - if (auto * low_cardinality_type = checkAndGetDataType(column.type.get())) - column.type = removeLowCardinality() ///low_cardinality_type->getDictionaryType(); + column.type = removeLowCardinality(column.type); } else if (auto * low_cardinality_column = checkAndGetColumn(column.column.get())) { From 7673d2abb5c2385d7e0fbd7d046d40fa455f786a Mon Sep 17 00:00:00 2001 From: KochetovNicolai Date: Tue, 12 Feb 2019 18:48:32 +0300 Subject: [PATCH 30/40] Update 906_low_cardinality_cache.sql Delete table after test. --- dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql b/dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql index cafc559d77f..ca286e9379f 100644 --- a/dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql +++ b/dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql @@ -2,4 +2,4 @@ drop table if exists test.lc; create table test.lc (b LowCardinality(String)) engine=MergeTree order by b; insert into test.lc select '0123456789' from numbers(100000000); select count(), b from test.lc group by b; - +drop table if exists test.lc; From 31397f715675b14acab4642a20fce77b6a1339ee Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Feb 2019 19:04:05 +0300 Subject: [PATCH 31/40] Fix column size for const LowCardinality arguments. --- dbms/src/Functions/IFunction.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/IFunction.cpp b/dbms/src/Functions/IFunction.cpp index 069a96f9ad8..85342fca358 100644 --- a/dbms/src/Functions/IFunction.cpp +++ b/dbms/src/Functions/IFunction.cpp @@ -332,9 +332,9 @@ static const ColumnLowCardinality * findLowCardinalityArgument(const Block & blo } static ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( - Block & block, const ColumnNumbers & args, bool can_be_executed_on_default_arguments) + Block & block, const ColumnNumbers & args, bool can_be_executed_on_default_arguments, size_t input_rows_count) { - size_t num_rows = 0; + size_t num_rows = input_rows_count; ColumnPtr indexes; for (auto arg : args) @@ -426,7 +426,7 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si block_without_low_cardinality.safeGetByPosition(result).type = res_low_cardinality_type->getDictionaryType(); ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( - block_without_low_cardinality, args, can_be_executed_on_default_arguments); + block_without_low_cardinality, args, can_be_executed_on_default_arguments, input_rows_count); executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, block_without_low_cardinality.rows(), dry_run); From 8e075899cbbbc081f71b6f26a0613fbd4ab750ac Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Feb 2019 19:15:20 +0300 Subject: [PATCH 32/40] Added test. --- .../0_stateless/00906_low_cardinality_const_argument.reference | 1 + .../0_stateless/00906_low_cardinality_const_argument.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00906_low_cardinality_const_argument.reference create mode 100644 dbms/tests/queries/0_stateless/00906_low_cardinality_const_argument.sql diff --git a/dbms/tests/queries/0_stateless/00906_low_cardinality_const_argument.reference b/dbms/tests/queries/0_stateless/00906_low_cardinality_const_argument.reference new file mode 100644 index 00000000000..78981922613 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00906_low_cardinality_const_argument.reference @@ -0,0 +1 @@ +a diff --git a/dbms/tests/queries/0_stateless/00906_low_cardinality_const_argument.sql b/dbms/tests/queries/0_stateless/00906_low_cardinality_const_argument.sql new file mode 100644 index 00000000000..831a4534f1e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00906_low_cardinality_const_argument.sql @@ -0,0 +1,2 @@ +select materialize(toLowCardinality('a')); + From 7d1e755a7c6b5b8903aed7e418e1c14f4bb37f3b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Feb 2019 19:41:06 +0300 Subject: [PATCH 33/40] Fixed rare race condition on startup of non-replicated MergeTree tables: concurrent attempt to remove a temporary directory [#CLICKHOUSE-4296] --- dbms/src/Storages/MergeTree/BackgroundProcessingPool.h | 2 ++ dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 6 ++++-- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h index b9c64aebfe9..ac7d231d966 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h @@ -53,7 +53,9 @@ public: return size; } + /// The task is started immediately. TaskHandle addTask(const Task & task); + void removeTask(const TaskHandle & task); ~BackgroundProcessingPool(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 689e166c757..2203d15a00f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -831,7 +831,7 @@ void MergeTreeData::clearOldTemporaryDirectories(ssize_t custom_directories_life Poco::DirectoryIterator end; for (Poco::DirectoryIterator it{full_path}; it != end; ++it) { - if (startsWith(it.name(), "tmp")) + if (startsWith(it.name(), "tmp_")) { Poco::File tmp_dir(full_path + it.name()); diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 0f9f2069645..0582cc859cb 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -87,13 +87,15 @@ StorageMergeTree::StorageMergeTree( void StorageMergeTree::startup() { - background_task_handle = background_pool.addTask([this] { return backgroundTask(); }); - data.clearOldPartsFromFilesystem(); /// Temporary directories contain incomplete results of merges (after forced restart) /// and don't allow to reinitialize them, so delete each of them immediately data.clearOldTemporaryDirectories(0); + + /// NOTE background task will also do the above cleanups periodically. + time_after_previous_cleanup.restart(); + background_task_handle = background_pool.addTask([this] { return backgroundTask(); }); } From 5279c7ff81b638fecbf193c9901e0e530ba70240 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 12 Feb 2019 20:54:52 +0300 Subject: [PATCH 34/40] setting --- dbms/src/Interpreters/Settings.h | 1 + dbms/src/Storages/IndicesDescription.h | 1 + dbms/src/Storages/MergeTree/MergeTreeData.cpp | 8 +++++++- dbms/src/Storages/MergeTree/MergeTreeData.h | 2 +- dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp | 4 ++++ dbms/src/Storages/StorageMergeTree.cpp | 2 +- dbms/src/Storages/StorageReplicatedMergeTree.cpp | 2 +- dbms/tests/queries/0_stateless/00836_indices_alter.sql | 2 ++ .../00836_indices_alter_replicated_zookeeper.sql | 2 ++ dbms/tests/queries/0_stateless/00837_minmax_index.sh | 9 ++++++--- .../00837_minmax_index_replicated_zookeeper.sql | 2 ++ dbms/tests/queries/0_stateless/00838_unique_index.sh | 4 +++- 12 files changed, 31 insertions(+), 8 deletions(-) diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 20b1e3cffaf..ef53f1300cb 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -300,6 +300,7 @@ struct Settings M(SettingBool, allow_experimental_cross_to_join_conversion, false, "Convert CROSS JOIN to INNER JOIN if possible") \ M(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.") \ M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only for 'mysql' table function.") \ + M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.")\ #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \ TYPE NAME {DEFAULT}; diff --git a/dbms/src/Storages/IndicesDescription.h b/dbms/src/Storages/IndicesDescription.h index 9d7c7907ca1..53b492198ce 100644 --- a/dbms/src/Storages/IndicesDescription.h +++ b/dbms/src/Storages/IndicesDescription.h @@ -14,6 +14,7 @@ struct IndicesDescription IndicesDescription() = default; + bool empty() const { return indices.empty(); } String toString() const; static IndicesDescription parse(const String & str); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 689e166c757..1df3f9f50a7 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -72,6 +72,7 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int MEMORY_LIMIT_EXCEEDED; extern const int SYNTAX_ERROR; extern const int INVALID_PARTITION_VALUE; @@ -1051,7 +1052,7 @@ bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to) } -void MergeTreeData::checkAlter(const AlterCommands & commands) +void MergeTreeData::checkAlter(const AlterCommands & commands, const Context & context) { /// Check that needed transformations can be applied to the list of columns without considering type conversions. auto new_columns = getColumns(); @@ -1060,6 +1061,11 @@ void MergeTreeData::checkAlter(const AlterCommands & commands) ASTPtr new_primary_key_ast = primary_key_ast; commands.apply(new_columns, new_indices, new_order_by_ast, new_primary_key_ast); + if (getIndicesDescription().empty() && !new_indices.empty() && + !context.getSettingsRef().allow_experimental_data_skipping_indices) + throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \ + "before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS); + /// Set of columns that shouldn't be altered. NameSet columns_alter_forbidden; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 66cbe6bf9dd..f252e43b562 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -481,7 +481,7 @@ public: /// - all type conversions can be done. /// - columns corresponding to primary key, indices, sign, sampling expression and date are not affected. /// If something is wrong, throws an exception. - void checkAlter(const AlterCommands & commands); + void checkAlter(const AlterCommands & commands, const Context & context); /// Performs ALTER of the data part, writes the result to temporary files. /// Returns an object allowing to rename temporary files to permanent files. diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 6fd61fb434c..1958b489023 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -610,6 +610,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) ErrorCodes::BAD_ARGUMENTS); } + if (!args.attach && !indices_description.empty() && !args.local_context.getSettingsRef().allow_experimental_data_skipping_indices) + throw Exception("You must set the setting `allow_experimental_data_skipping_indices` to 1 " \ + "before using data skipping indices.", ErrorCodes::BAD_ARGUMENTS); + if (replicated) return StorageReplicatedMergeTree::create( zookeeper_path, replica_name, args.attach, args.data_path, args.database_name, args.table_name, diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 0f9f2069645..7f4c844aa09 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -211,7 +211,7 @@ void StorageMergeTree::alter( auto table_soft_lock = lockDataForAlter(); - data.checkAlter(params); + data.checkAlter(params, context); auto new_columns = data.getColumns(); auto new_indices = data.getIndicesDescription(); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 062800871e6..a1eaee63a72 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -3125,7 +3125,7 @@ void StorageReplicatedMergeTree::alter(const AlterCommands & params, if (is_readonly) throw Exception("Can't ALTER readonly table", ErrorCodes::TABLE_IS_READ_ONLY); - data.checkAlter(params); + data.checkAlter(params, query_context); ColumnsDescription new_columns = data.getColumns(); IndicesDescription new_indices = data.getIndicesDescription(); diff --git a/dbms/tests/queries/0_stateless/00836_indices_alter.sql b/dbms/tests/queries/0_stateless/00836_indices_alter.sql index 6749d69ff28..0fb8d49da6a 100644 --- a/dbms/tests/queries/0_stateless/00836_indices_alter.sql +++ b/dbms/tests/queries/0_stateless/00836_indices_alter.sql @@ -1,6 +1,8 @@ DROP TABLE IF EXISTS test.minmax_idx; DROP TABLE IF EXISTS test.minmax_idx2; +SET allow_experimental_data_skipping_indices = 1; + CREATE TABLE test.minmax_idx ( u64 UInt64, diff --git a/dbms/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql b/dbms/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql index ccd71beb6ec..4240348f7de 100644 --- a/dbms/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00836_indices_alter_replicated_zookeeper.sql @@ -3,6 +3,8 @@ DROP TABLE IF EXISTS test.minmax_idx_r; DROP TABLE IF EXISTS test.minmax_idx2; DROP TABLE IF EXISTS test.minmax_idx2_r; +SET allow_experimental_data_skipping_indices = 1; + CREATE TABLE test.minmax_idx ( u64 UInt64, diff --git a/dbms/tests/queries/0_stateless/00837_minmax_index.sh b/dbms/tests/queries/0_stateless/00837_minmax_index.sh index 72c36be1bf1..d38f7bbabfd 100755 --- a/dbms/tests/queries/0_stateless/00837_minmax_index.sh +++ b/dbms/tests/queries/0_stateless/00837_minmax_index.sh @@ -3,9 +3,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.minmax_idx + +$CLICKHOUSE_CLIENT -n --query=" +SET allow_experimental_data_skipping_indices = 1; +CREATE TABLE test.minmax_idx ( u64 UInt64, i32 Int32, @@ -19,7 +22,7 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE test.minmax_idx INDEX idx_2 (u64 + toYear(dt), substring(s, 2, 4)) TYPE minmax GRANULARITY 3 ) ENGINE = MergeTree() ORDER BY u64 -SETTINGS index_granularity = 2" +SETTINGS index_granularity = 2;" $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES diff --git a/dbms/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql b/dbms/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql index 8d0d5ce80ff..ceb361170f3 100644 --- a/dbms/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00837_minmax_index_replicated_zookeeper.sql @@ -1,6 +1,8 @@ DROP TABLE IF EXISTS test.minmax_idx1; DROP TABLE IF EXISTS test.minmax_idx2; +SET allow_experimental_data_skipping_indices = 1; + CREATE TABLE test.minmax_idx1 ( u64 UInt64, diff --git a/dbms/tests/queries/0_stateless/00838_unique_index.sh b/dbms/tests/queries/0_stateless/00838_unique_index.sh index 9f33c916689..dd4440bd5ce 100755 --- a/dbms/tests/queries/0_stateless/00838_unique_index.sh +++ b/dbms/tests/queries/0_stateless/00838_unique_index.sh @@ -5,7 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.set_idx;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.set_idx +$CLICKHOUSE_CLIENT -n --query=" +SET allow_experimental_data_skipping_indices = 1; +CREATE TABLE test.set_idx ( u64 UInt64, i32 Int32, From dddec4073ea0ccd8cf9a8a8549bc4d9bfff3f208 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 12 Feb 2019 20:56:57 +0300 Subject: [PATCH 35/40] fix --- docs/en/operations/table_engines/mergetree.md | 2 +- docs/ru/operations/table_engines/mergetree.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index 35c7feb36b9..29647a76e33 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -228,7 +228,7 @@ To check whether ClickHouse can use the index when running a query, use the sett The key for partitioning by month allows reading only those data blocks which contain dates from the proper range. In this case, the data block may contain data for many dates (up to an entire month). Within a block, data is sorted by primary key, which might not contain the date as the first column. Because of this, using a query with only a date condition that does not specify the primary key prefix will cause more data to be read than for a single date. -### Data Skipping Indices +### Data Skipping Indices (Experimental) Index declaration in the columns section of the `CREATE` query. ```sql diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index aa288e45434..f27dacb3495 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -226,7 +226,7 @@ SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' Ключ партиционирования по месяцам обеспечивает чтение только тех блоков данных, которые содержат даты из нужного диапазона. При этом блок данных может содержать данные за многие даты (до целого месяца). В пределах одного блока данные упорядочены по первичному ключу, который может не содержать дату в качестве первого столбца. В связи с этим, при использовании запроса с указанием условия только на дату, но не на префикс первичного ключа, будет читаться данных больше, чем за одну дату. -### Дополнительные индексы +### Дополнительные индексы (Экспериментальный функционал) Для таблиц семейства `*MergeTree` можно задать дополнительные индексы в секции столбцов. From a27c61b0656294f4614f911398b72eb4fd472fdd Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 12 Feb 2019 21:02:45 +0300 Subject: [PATCH 36/40] docs --- docs/en/operations/table_engines/mergetree.md | 2 ++ docs/ru/operations/table_engines/mergetree.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index 29647a76e33..948d63ff7d8 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -230,6 +230,8 @@ The key for partitioning by month allows reading only those data blocks which co ### Data Skipping Indices (Experimental) +You need to set `allow_experimental_data_skipping_indices` to 1 to use indices. (run `SET allow_experimental_data_skipping_indices = 1`). + Index declaration in the columns section of the `CREATE` query. ```sql INDEX index_name expr TYPE type(...) GRANULARITY granularity_value diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index f27dacb3495..3c4f84d1c8c 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -228,6 +228,8 @@ SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ### Дополнительные индексы (Экспериментальный функционал) +Для использования требуется установить настройку `allow_experimental_data_skipping_indices` в 1. (запустить `SET allow_experimental_data_skipping_indices = 1`). + Для таблиц семейства `*MergeTree` можно задать дополнительные индексы в секции столбцов. Индексы аггрегируют для заданного выражения некоторые данные, а потом при `SELECT` запросе используют для пропуска боков данных (пропускаемый блок состоих из гранул данных в количестве равном гранулярности данного индекса), на которых секция `WHERE` не может быть выполнена, тем самым уменьшая объем данных читаемых с диска. From 0bf4f4334bc46b324fc4582e826e09d506ef1b3b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 12 Feb 2019 21:27:14 +0300 Subject: [PATCH 37/40] Renamed tests. --- ...lity_cache.reference => 00906_low_cardinality_cache.reference} | 0 ..._low_cardinality_cache.sql => 00906_low_cardinality_cache.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{906_low_cardinality_cache.reference => 00906_low_cardinality_cache.reference} (100%) rename dbms/tests/queries/0_stateless/{906_low_cardinality_cache.sql => 00906_low_cardinality_cache.sql} (100%) diff --git a/dbms/tests/queries/0_stateless/906_low_cardinality_cache.reference b/dbms/tests/queries/0_stateless/00906_low_cardinality_cache.reference similarity index 100% rename from dbms/tests/queries/0_stateless/906_low_cardinality_cache.reference rename to dbms/tests/queries/0_stateless/00906_low_cardinality_cache.reference diff --git a/dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql b/dbms/tests/queries/0_stateless/00906_low_cardinality_cache.sql similarity index 100% rename from dbms/tests/queries/0_stateless/906_low_cardinality_cache.sql rename to dbms/tests/queries/0_stateless/00906_low_cardinality_cache.sql From 7665ef297fba68fdb7762197b8b05f580bf61d53 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 12 Feb 2019 21:42:14 +0300 Subject: [PATCH 38/40] Auto version update to [19.3.1] [54415] --- dbms/cmake/version.cmake | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index f19b898939b..5f3758b89a6 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -2,10 +2,10 @@ set(VERSION_REVISION 54415) set(VERSION_MAJOR 19) set(VERSION_MINOR 3) -set(VERSION_PATCH 0) -set(VERSION_GITHASH 1db4bd8c2a1a0cd610c8a6564e8194dca5265562) -set(VERSION_DESCRIBE v19.3.0-testing) -set(VERSION_STRING 19.3.0) +set(VERSION_PATCH 1) +set(VERSION_GITHASH 48280074c4a9151ca010fb0a777efd82634460bd) +set(VERSION_DESCRIBE v19.3.1-testing) +set(VERSION_STRING 19.3.1) # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/debian/changelog b/debian/changelog index f1e2c9cd754..8d988cb3cac 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (19.3.0) unstable; urgency=low +clickhouse (19.3.1) unstable; urgency=low * Modified source code - -- Mon, 11 Feb 2019 18:13:23 +0300 + -- Tue, 12 Feb 2019 21:42:14 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index e817274d1e5..2196a342ecd 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.3.0 +ARG version=19.3.1 RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 93d19b3ee60..244c86f92a9 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.3.0 +ARG version=19.3.1 ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index ef7299c5b7c..e5cf7036165 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.3.0 +ARG version=19.3.1 RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ From 67e3bc501bbe2a56bd4f432840b5048425ca0090 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Feb 2019 22:19:12 +0300 Subject: [PATCH 39/40] Add brotli to stateless test image --- docker/test/stateless/Dockerfile | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 9faf98bf066..34a64a24b1a 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -19,7 +19,9 @@ RUN apt-get update -y \ openssl \ netcat-openbsd \ telnet \ - moreutils + moreutils \ + brotli + ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone From c8cafa456c51a3c1a35360228ad9e066a25bf41d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 12 Feb 2019 22:23:22 +0300 Subject: [PATCH 40/40] Added a test for already fixed issue [#CLICKHOUSE-4260] --- ...optimize_predicate_and_rename_table.reference | 2 ++ ...00843_optimize_predicate_and_rename_table.sql | 16 ++++++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00843_optimize_predicate_and_rename_table.reference create mode 100644 dbms/tests/queries/0_stateless/00843_optimize_predicate_and_rename_table.sql diff --git a/dbms/tests/queries/0_stateless/00843_optimize_predicate_and_rename_table.reference b/dbms/tests/queries/0_stateless/00843_optimize_predicate_and_rename_table.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00843_optimize_predicate_and_rename_table.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/dbms/tests/queries/0_stateless/00843_optimize_predicate_and_rename_table.sql b/dbms/tests/queries/0_stateless/00843_optimize_predicate_and_rename_table.sql new file mode 100644 index 00000000000..324f9e7c5e0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00843_optimize_predicate_and_rename_table.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS test.test1; +DROP TABLE IF EXISTS test.test2; +DROP TABLE IF EXISTS test.view; + +CREATE TABLE test.test1 (a UInt8) ENGINE = Memory; +INSERT INTO test.test1 VALUES (1); + +CREATE VIEW test.view AS SELECT * FROM test.test1; +SELECT * FROM test.view; +RENAME TABLE test.test1 TO test.test2; +SELECT * FROM test.view; -- { serverError 60 } +RENAME TABLE test.test2 TO test.test1; +SELECT * FROM test.view; + +DROP TABLE test.test1; +DROP TABLE test.view;