From 0850d63d52a2b8a222a5a5d00d96967deb03668b Mon Sep 17 00:00:00 2001 From: Alexey Ilyukhov Date: Tue, 5 May 2020 17:40:04 +0300 Subject: [PATCH 001/150] Add geo data types point and polygon --- src/DataTypes/DataTypeCustomGeo.cpp | 74 +++++++++++++++++++++++++++++ src/DataTypes/DataTypeFactory.cpp | 1 + src/DataTypes/DataTypeFactory.h | 1 + src/DataTypes/ya.make | 1 + 4 files changed, 77 insertions(+) create mode 100644 src/DataTypes/DataTypeCustomGeo.cpp diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp new file mode 100644 index 00000000000..24ab57626cd --- /dev/null +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -0,0 +1,74 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +class DataTypeCustomPointSerialization : public DataTypeCustomSimpleTextSerialization +{ +private: + DataTypePtr tuple; + +public: + DataTypeCustomPointSerialization() : tuple(std::make_unique( + DataTypes({std::make_unique(), std::make_unique()}))) + {} + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override + { + tuple->serializeAsText(column, row_num, ostr, settings); + } + + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + tuple->deserializeAsWholeText(column, istr, settings); + } +}; + +class DataTypeCustomPolygonSerialization : public DataTypeCustomSimpleTextSerialization +{ +private: + DataTypePtr array; + +public: + DataTypeCustomPolygonSerialization() : array(std::make_unique(std::make_unique(std::make_unique( + DataTypes({std::make_unique(), std::make_unique()}))))) + {} + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override + { + array->serializeAsText(column, row_num, ostr, settings); + } + + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + array->deserializeAsWholeText(column, istr, settings); + } +}; + +} + +void registerDataTypeDomainGeo(DataTypeFactory & factory) { + factory.registerSimpleDataTypeCustom("Point", [] + { + return std::make_pair(DataTypeFactory::instance().get("Tuple(Float64, Float64)"), + std::make_unique(std::make_unique("Point"), std::make_unique())); + }); + + factory.registerSimpleDataTypeCustom("Polygon", [] + { + return std::make_pair(DataTypeFactory::instance().get("Array(Array(Tuple(Float64, Float64)))"), + std::make_unique(std::make_unique("Polygon"), std::make_unique())); + }); +} + +} \ No newline at end of file diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index f81adfe347c..880f25d009d 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -180,6 +180,7 @@ DataTypeFactory::DataTypeFactory() registerDataTypeLowCardinality(*this); registerDataTypeDomainIPv4AndIPv6(*this); registerDataTypeDomainSimpleAggregateFunction(*this); + registerDataTypeDomainGeo(*this); } DataTypeFactory & DataTypeFactory::instance() diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index 8844e95d1bc..6bf09d31727 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -83,5 +83,6 @@ void registerDataTypeLowCardinality(DataTypeFactory & factory); void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); void registerDataTypeDateTime64(DataTypeFactory & factory); +void registerDataTypeDomainGeo(DataTypeFactory & factory); } diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index c3a1e452d0d..6e80ed0b525 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -10,6 +10,7 @@ SRCS( DataTypeAggregateFunction.cpp DataTypeArray.cpp DataTypeCustomIPv4AndIPv6.cpp + DataTypeCustomPoint.cpp DataTypeCustomSimpleAggregateFunction.cpp DataTypeCustomSimpleTextSerialization.cpp DataTypeDate.cpp From d6b57ff32f3b1e694f9a0ef1993ec79348209af8 Mon Sep 17 00:00:00 2001 From: DoomzD Date: Tue, 5 May 2020 18:36:50 +0300 Subject: [PATCH 002/150] Change Tuple(F64, F64) to Point in Polygon definition --- src/DataTypes/DataTypeCustomGeo.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index 24ab57626cd..db157bd81db 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -66,7 +66,7 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory) { factory.registerSimpleDataTypeCustom("Polygon", [] { - return std::make_pair(DataTypeFactory::instance().get("Array(Array(Tuple(Float64, Float64)))"), + return std::make_pair(DataTypeFactory::instance().get("Array(Array(Point))"), std::make_unique(std::make_unique("Polygon"), std::make_unique())); }); } From f3e194307e982e2e7466098550f6ededf4586910 Mon Sep 17 00:00:00 2001 From: DoomzD Date: Tue, 5 May 2020 19:47:22 +0300 Subject: [PATCH 003/150] Add MultiPolygon datatype --- src/DataTypes/DataTypeCustomGeo.cpp | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index db157bd81db..8276afd77d7 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -55,6 +55,29 @@ public: } }; +class DataTypeCustomMultiPolygonSerialization : public DataTypeCustomSimpleTextSerialization +{ +private: + DataTypePtr array; + +public: + DataTypeCustomMultiPolygonSerialization() : array( + std::make_unique(std::make_unique( + std::make_unique(std::make_unique( + DataTypes({std::make_unique(), std::make_unique()})))))) + {} + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override + { + array->serializeAsText(column, row_num, ostr, settings); + } + + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + array->deserializeAsWholeText(column, istr, settings); + } +}; + } void registerDataTypeDomainGeo(DataTypeFactory & factory) { @@ -69,6 +92,12 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory) { return std::make_pair(DataTypeFactory::instance().get("Array(Array(Point))"), std::make_unique(std::make_unique("Polygon"), std::make_unique())); }); + + factory.registerSimpleDataTypeCustom("MultiPolygon", [] + { + return std::make_pair(DataTypeFactory::instance().get("Array(Polygon)"), + std::make_unique(std::make_unique("MultiPolygon"), std::make_unique())); + }); } } \ No newline at end of file From ad9944da62a34fb3ef35aa67cea4e1e19910928b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 10 May 2020 03:02:30 +0300 Subject: [PATCH 004/150] Update DataTypeCustomGeo.cpp --- src/DataTypes/DataTypeCustomGeo.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index 8276afd77d7..c8df3b5a8d9 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -100,4 +100,4 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory) { }); } -} \ No newline at end of file +} From ccd88d8bc8edda1114fbdb558c2f4f39b4422e0d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 11 May 2020 06:31:28 +0300 Subject: [PATCH 005/150] Fix style. --- src/DataTypes/DataTypeCustomGeo.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index c8df3b5a8d9..c25880fbcce 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -80,7 +80,8 @@ public: } -void registerDataTypeDomainGeo(DataTypeFactory & factory) { +void registerDataTypeDomainGeo(DataTypeFactory & factory) +{ factory.registerSimpleDataTypeCustom("Point", [] { return std::make_pair(DataTypeFactory::instance().get("Tuple(Float64, Float64)"), From a65923b028aecfac6930906bad98ae509cce4cf7 Mon Sep 17 00:00:00 2001 From: Alexey Ilyukhov Date: Sun, 17 May 2020 17:02:41 +0300 Subject: [PATCH 006/150] Add comments --- src/DataTypes/DataTypeCustomGeo.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index c25880fbcce..d3d808a1b50 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -82,18 +82,23 @@ public: void registerDataTypeDomainGeo(DataTypeFactory & factory) { + // Custom type for point represented as its coordinates stored as Tuple(Float64, Float64) factory.registerSimpleDataTypeCustom("Point", [] { return std::make_pair(DataTypeFactory::instance().get("Tuple(Float64, Float64)"), std::make_unique(std::make_unique("Point"), std::make_unique())); }); + // Custom type for polygon with holes stored as Array(Array(Point)) + // Each element of outer array represents a simple polygon without holes stored as array of points + // First element of outer array is outer shape of polygon and all the following are holes factory.registerSimpleDataTypeCustom("Polygon", [] { return std::make_pair(DataTypeFactory::instance().get("Array(Array(Point))"), std::make_unique(std::make_unique("Polygon"), std::make_unique())); }); + // Custom type for multiple polygons with holes stored as Array(Polygon) factory.registerSimpleDataTypeCustom("MultiPolygon", [] { return std::make_pair(DataTypeFactory::instance().get("Array(Polygon)"), From f026fa7de1109c3c0c7e5ad2f643c9c3267f7b42 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 22 May 2020 16:29:33 +0300 Subject: [PATCH 007/150] Rename PartDestinationType to DataDestionationType, because there is no part-specific information --- src/Interpreters/InterpreterAlterQuery.cpp | 6 ++--- src/Parsers/ASTAlterQuery.cpp | 8 +++---- src/Parsers/ASTAlterQuery.h | 2 +- src/Parsers/ASTTTLElement.cpp | 6 ++--- src/Parsers/ASTTTLElement.h | 6 ++--- src/Parsers/ExpressionElementParsers.cpp | 8 +++---- src/Parsers/ParserAlterQuery.cpp | 16 +++++++------- ...estinationType.h => DataDestinationType.h} | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 22 +++++++++---------- src/Storages/MergeTree/MergeTreeData.h | 4 ++-- src/Storages/PartitionCommands.cpp | 8 +++---- src/Storages/StorageInMemoryMetadata.h | 13 +++++++++++ 12 files changed, 57 insertions(+), 44 deletions(-) rename src/Storages/{MergeTree/PartDestinationType.h => DataDestinationType.h} (73%) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c2ab8776c25..c3ee740a486 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -244,12 +244,12 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS } case ASTAlterCommand::MOVE_PARTITION: { - if ((command.move_destination_type == PartDestinationType::DISK) - || (command.move_destination_type == PartDestinationType::VOLUME)) + if ((command.move_destination_type == DataDestinationType::DISK) + || (command.move_destination_type == DataDestinationType::VOLUME)) { required_access.emplace_back(AccessType::ALTER_MOVE_PARTITION, database, table); } - else if (command.move_destination_type == PartDestinationType::TABLE) + else if (command.move_destination_type == DataDestinationType::TABLE) { required_access.emplace_back(AccessType::SELECT | AccessType::ALTER_DELETE, database, table); required_access.emplace_back(AccessType::INSERT, command.to_database, command.to_table); diff --git a/src/Parsers/ASTAlterQuery.cpp b/src/Parsers/ASTAlterQuery.cpp index 9ec2fad5768..f323f66be17 100644 --- a/src/Parsers/ASTAlterQuery.cpp +++ b/src/Parsers/ASTAlterQuery.cpp @@ -181,13 +181,13 @@ void ASTAlterCommand::formatImpl( settings.ostr << " TO "; switch (move_destination_type) { - case PartDestinationType::DISK: + case DataDestinationType::DISK: settings.ostr << "DISK "; break; - case PartDestinationType::VOLUME: + case DataDestinationType::VOLUME: settings.ostr << "VOLUME "; break; - case PartDestinationType::TABLE: + case DataDestinationType::TABLE: settings.ostr << "TABLE "; if (!to_database.empty()) { @@ -201,7 +201,7 @@ void ASTAlterCommand::formatImpl( default: break; } - if (move_destination_type != PartDestinationType::TABLE) + if (move_destination_type != DataDestinationType::TABLE) { settings.ostr << quoteString(move_destination_name); } diff --git a/src/Parsers/ASTAlterQuery.h b/src/Parsers/ASTAlterQuery.h index 85e9a4d7552..a9ae06863a9 100644 --- a/src/Parsers/ASTAlterQuery.h +++ b/src/Parsers/ASTAlterQuery.h @@ -136,7 +136,7 @@ public: bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN - PartDestinationType move_destination_type; /// option for MOVE PART/PARTITION + DataDestinationType move_destination_type; /// option for MOVE PART/PARTITION String move_destination_name; /// option for MOVE PART/PARTITION diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 7e03a73e36d..acc3546ddfb 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -10,15 +10,15 @@ namespace DB void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { children.front()->formatImpl(settings, state, frame); - if (destination_type == PartDestinationType::DISK) + if (destination_type == DataDestinationType::DISK) { settings.ostr << " TO DISK " << quoteString(destination_name); } - else if (destination_type == PartDestinationType::VOLUME) + else if (destination_type == DataDestinationType::VOLUME) { settings.ostr << " TO VOLUME " << quoteString(destination_name); } - else if (destination_type == PartDestinationType::DELETE) + else if (destination_type == DataDestinationType::DELETE) { /// It would be better to output "DELETE" here but that will break compatibility with earlier versions. } diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index 02f70094e04..61f4a6e958f 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -11,10 +11,10 @@ namespace DB class ASTTTLElement : public IAST { public: - PartDestinationType destination_type; + DataDestinationType destination_type; String destination_name; - ASTTTLElement(PartDestinationType destination_type_, const String & destination_name_) + ASTTTLElement(DataDestinationType destination_type_, const String & destination_name_) : destination_type(destination_type_) , destination_name(destination_name_) { diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 70a8b282a72..c206db24626 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1462,16 +1462,16 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parser_exp.parse(pos, expr_elem, expected)) return false; - PartDestinationType destination_type = PartDestinationType::DELETE; + DataDestinationType destination_type = DataDestinationType::DELETE; String destination_name; if (s_to_disk.ignore(pos)) - destination_type = PartDestinationType::DISK; + destination_type = DataDestinationType::DISK; else if (s_to_volume.ignore(pos)) - destination_type = PartDestinationType::VOLUME; + destination_type = DataDestinationType::VOLUME; else s_delete.ignore(pos); - if (destination_type == PartDestinationType::DISK || destination_type == PartDestinationType::VOLUME) + if (destination_type == DataDestinationType::DISK || destination_type == DataDestinationType::VOLUME) { ASTPtr ast_space_name; if (!parser_string_literal.parse(pos, ast_space_name, expected)) diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 623bca440bb..f90d010e9de 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -260,19 +260,19 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->part = true; if (s_to_disk.ignore(pos)) - command->move_destination_type = PartDestinationType::DISK; + command->move_destination_type = DataDestinationType::DISK; else if (s_to_volume.ignore(pos)) - command->move_destination_type = PartDestinationType::VOLUME; + command->move_destination_type = DataDestinationType::VOLUME; else if (s_to_table.ignore(pos)) { if (!parseDatabaseAndTableName(pos, expected, command->to_database, command->to_table)) return false; - command->move_destination_type = PartDestinationType::TABLE; + command->move_destination_type = DataDestinationType::TABLE; } else return false; - if (command->move_destination_type != PartDestinationType::TABLE) + if (command->move_destination_type != DataDestinationType::TABLE) { ASTPtr ast_space_name; if (!parser_string_literal.parse(pos, ast_space_name, expected)) @@ -289,19 +289,19 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->type = ASTAlterCommand::MOVE_PARTITION; if (s_to_disk.ignore(pos)) - command->move_destination_type = PartDestinationType::DISK; + command->move_destination_type = DataDestinationType::DISK; else if (s_to_volume.ignore(pos)) - command->move_destination_type = PartDestinationType::VOLUME; + command->move_destination_type = DataDestinationType::VOLUME; else if (s_to_table.ignore(pos)) { if (!parseDatabaseAndTableName(pos, expected, command->to_database, command->to_table)) return false; - command->move_destination_type = PartDestinationType::TABLE; + command->move_destination_type = DataDestinationType::TABLE; } else return false; - if (command->move_destination_type != PartDestinationType::TABLE) + if (command->move_destination_type != DataDestinationType::TABLE) { ASTPtr ast_space_name; if (!parser_string_literal.parse(pos, ast_space_name, expected)) diff --git a/src/Storages/MergeTree/PartDestinationType.h b/src/Storages/DataDestinationType.h similarity index 73% rename from src/Storages/MergeTree/PartDestinationType.h rename to src/Storages/DataDestinationType.h index 8f3e44537a6..05d1d89c2b5 100644 --- a/src/Storages/MergeTree/PartDestinationType.h +++ b/src/Storages/DataDestinationType.h @@ -4,7 +4,7 @@ namespace DB { -enum class PartDestinationType +enum class DataDestinationType { DISK, VOLUME, diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 728bfa18bd8..ef8e3789cb1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -621,7 +621,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, auto syntax_result = SyntaxAnalyzer(global_context).analyze(ttl_ast, new_columns.getAllPhysical()); result.expression = ExpressionAnalyzer(ttl_ast, syntax_result, global_context).getActions(false); - result.destination_type = PartDestinationType::DELETE; + result.destination_type = DataDestinationType::DELETE; result.result_column = ttl_ast->getColumnName(); checkTTLExpression(result.expression, result.result_column); @@ -665,7 +665,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, if (!ttl_element) throw Exception("Unexpected AST element in TTL expression", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - if (ttl_element->destination_type == PartDestinationType::DELETE) + if (ttl_element->destination_type == DataDestinationType::DELETE) { if (seen_delete_ttl) { @@ -688,7 +688,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, if (!new_ttl_entry.getDestination(getStoragePolicy())) { String message; - if (new_ttl_entry.destination_type == PartDestinationType::DISK) + if (new_ttl_entry.destination_type == DataDestinationType::DISK) message = "No such disk " + backQuote(new_ttl_entry.destination_name) + " for given storage policy."; else message = "No such volume " + backQuote(new_ttl_entry.destination_name) + " for given storage policy."; @@ -2921,11 +2921,11 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ SpacePtr destination_ptr = ttl_entry->getDestination(getStoragePolicy()); if (!destination_ptr) { - if (ttl_entry->destination_type == PartDestinationType::VOLUME) + if (ttl_entry->destination_type == DataDestinationType::VOLUME) LOG_WARNING(log, "Would like to reserve space on volume '" << ttl_entry->destination_name << "' by TTL rule of table '" << log_name << "' but volume was not found"); - else if (ttl_entry->destination_type == PartDestinationType::DISK) + else if (ttl_entry->destination_type == DataDestinationType::DISK) LOG_WARNING(log, "Would like to reserve space on disk '" << ttl_entry->destination_name << "' by TTL rule of table '" << log_name << "' but disk was not found"); @@ -2936,11 +2936,11 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ if (reservation) return reservation; else - if (ttl_entry->destination_type == PartDestinationType::VOLUME) + if (ttl_entry->destination_type == DataDestinationType::VOLUME) LOG_WARNING(log, "Would like to reserve space on volume '" << ttl_entry->destination_name << "' by TTL rule of table '" << log_name << "' but there is not enough space"); - else if (ttl_entry->destination_type == PartDestinationType::DISK) + else if (ttl_entry->destination_type == DataDestinationType::DISK) LOG_WARNING(log, "Would like to reserve space on disk '" << ttl_entry->destination_name << "' by TTL rule of table '" << log_name << "' but there is not enough space"); @@ -2954,9 +2954,9 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ SpacePtr MergeTreeData::TTLEntry::getDestination(StoragePolicyPtr policy) const { - if (destination_type == PartDestinationType::VOLUME) + if (destination_type == DataDestinationType::VOLUME) return policy->getVolumeByName(destination_name); - else if (destination_type == PartDestinationType::DISK) + else if (destination_type == DataDestinationType::DISK) return policy->getDiskByName(destination_name); else return {}; @@ -2964,13 +2964,13 @@ SpacePtr MergeTreeData::TTLEntry::getDestination(StoragePolicyPtr policy) const bool MergeTreeData::TTLEntry::isPartInDestination(StoragePolicyPtr policy, const IMergeTreeDataPart & part) const { - if (destination_type == PartDestinationType::VOLUME) + if (destination_type == DataDestinationType::VOLUME) { for (const auto & disk : policy->getVolumeByName(destination_name)->getDisks()) if (disk->getName() == part.volume->getDisk()->getName()) return true; } - else if (destination_type == PartDestinationType::DISK) + else if (destination_type == DataDestinationType::DISK) return policy->getDiskByName(destination_name)->getName() == part.volume->getDisk()->getName(); return false; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 70d3a1c3514..2b5b75110f1 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include @@ -652,7 +652,7 @@ public: String result_column; /// Name and type of a destination are only valid in table-level context. - PartDestinationType destination_type; + DataDestinationType destination_type; String destination_name; ASTPtr entry_ast; diff --git a/src/Storages/PartitionCommands.cpp b/src/Storages/PartitionCommands.cpp index 9c8fc2cb598..c3bf00187af 100644 --- a/src/Storages/PartitionCommands.cpp +++ b/src/Storages/PartitionCommands.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include @@ -42,13 +42,13 @@ std::optional PartitionCommand::parse(const ASTAlterCommand * res.part = command_ast->part; switch (command_ast->move_destination_type) { - case PartDestinationType::DISK: + case DataDestinationType::DISK: res.move_destination_type = PartitionCommand::MoveDestinationType::DISK; break; - case PartDestinationType::VOLUME: + case DataDestinationType::VOLUME: res.move_destination_type = PartitionCommand::MoveDestinationType::VOLUME; break; - case PartDestinationType::TABLE: + case DataDestinationType::TABLE: res.move_destination_type = PartitionCommand::MoveDestinationType::TABLE; res.to_database = command_ast->to_database; res.to_table = command_ast->to_table; diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 37dce5a78b9..3faaed8d680 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -72,5 +73,17 @@ struct StorageMetadataKeyField static StorageMetadataKeyField getKeyFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context); }; +struct StorageMetadataTTLField +{ + ASTPtr definition_ast; + + ExpressionActionsPtr expression; + + String result_column; + + DataDestinationType destination_type; + + String destination_name; +}; } From 6d40c1ea2c2f149ab374f19e37e96f8e99c85022 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 2 May 2020 01:21:09 +0300 Subject: [PATCH 008/150] Preserve order of expressions during predicates optimization (enable_optimize_predicate_expression) Before this patch enable_optimize_predicate_expression changes the order *every* time, so: foo AND bar -> bar AND foo And this causes troubles for distributed queries, since query on shard will have different order of expressions with the initiating server. --- src/Interpreters/PredicateExpressionsOptimizer.cpp | 12 +++--------- .../00740_optimize_predicate_expression.reference | 1 + .../00740_optimize_predicate_expression.sql | 4 ++++ 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/PredicateExpressionsOptimizer.cpp b/src/Interpreters/PredicateExpressionsOptimizer.cpp index d2f38b85b4b..b5d2c632135 100644 --- a/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -49,13 +49,6 @@ static ASTs splitConjunctionPredicate(const std::initializer_list { std::vector res; - auto remove_expression_at_index = [&res] (const size_t index) - { - if (index < res.size() - 1) - std::swap(res[index], res.back()); - res.pop_back(); - }; - for (const auto & predicate : predicates) { if (!predicate) @@ -65,14 +58,15 @@ static ASTs splitConjunctionPredicate(const std::initializer_list for (size_t idx = 0; idx < res.size();) { - const auto & expression = res.at(idx); + ASTPtr expression = res.at(idx); if (const auto * function = expression->as(); function && function->name == "and") { + res.erase(res.begin() + idx); + for (auto & child : function->arguments->children) res.emplace_back(child); - remove_expression_at_index(idx); continue; } ++idx; diff --git a/tests/queries/0_stateless/00740_optimize_predicate_expression.reference b/tests/queries/0_stateless/00740_optimize_predicate_expression.reference index 6db331af725..6ec592953a9 100644 --- a/tests/queries/0_stateless/00740_optimize_predicate_expression.reference +++ b/tests/queries/0_stateless/00740_optimize_predicate_expression.reference @@ -1 +1,2 @@ nan +SELECT dummy\nFROM system.one\nWHERE (dummy > 0) AND (dummy < 0) diff --git a/tests/queries/0_stateless/00740_optimize_predicate_expression.sql b/tests/queries/0_stateless/00740_optimize_predicate_expression.sql index b016ab49ddd..3be996b265c 100644 --- a/tests/queries/0_stateless/00740_optimize_predicate_expression.sql +++ b/tests/queries/0_stateless/00740_optimize_predicate_expression.sql @@ -32,4 +32,8 @@ FROM ( WHERE user_id = 999 ) js2 USING site); +-- check order is preserved +SET enable_debug_queries = 1; +ANALYZE SELECT * FROM system.one HAVING dummy > 0 AND dummy < 0; + DROP TABLE perf; From f19c619dfe87a5254add11a76a127248601eea29 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 2 May 2020 01:32:57 +0300 Subject: [PATCH 009/150] Add test from #10613 for enable_optimize_predicate_expression --- .../0_stateless/00740_optimize_predicate_expression.sql | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/tests/queries/0_stateless/00740_optimize_predicate_expression.sql b/tests/queries/0_stateless/00740_optimize_predicate_expression.sql index 3be996b265c..78065c9896e 100644 --- a/tests/queries/0_stateless/00740_optimize_predicate_expression.sql +++ b/tests/queries/0_stateless/00740_optimize_predicate_expression.sql @@ -36,4 +36,11 @@ FROM ( SET enable_debug_queries = 1; ANALYZE SELECT * FROM system.one HAVING dummy > 0 AND dummy < 0; +-- from #10613 +SELECT name, count() AS cnt +FROM remote('127.{1,2}', system.settings) +GROUP BY name +HAVING (max(value) > '9') AND (min(changed) = 0) +FORMAT Null; + DROP TABLE perf; From 765ed4f89b80eeb63102ae8f1bb9ad60dba9a3fe Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 2 May 2020 13:33:34 +0300 Subject: [PATCH 010/150] Fix 01056_predicate_optimizer_bugs --- .../0_stateless/01056_predicate_optimizer_bugs.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 019e95cb359..3fbdd87eec4 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -4,7 +4,7 @@ a 2 1 0 a 3 1 0 b 13 2 0 b 15 2 0 -SELECT \n co, \n co2, \n co3, \n num\nFROM \n(\n SELECT \n co, \n co2, \n co3, \n count() AS num\n FROM \n (\n SELECT \n 1 AS co, \n 2 AS co2, \n 3 AS co3\n )\n GROUP BY \n co, \n co2, \n co3\n WITH CUBE\n HAVING (co != 0) AND (co2 != 2)\n)\nWHERE (co != 0) AND (co2 != 2) +SELECT \n co, \n co2, \n co3, \n num\nFROM \n(\n SELECT \n co, \n co2, \n co3, \n count() AS num\n FROM \n (\n SELECT \n 1 AS co, \n 2 AS co2, \n 3 AS co3\n )\n GROUP BY \n co, \n co2, \n co3\n WITH CUBE\n HAVING (co2 != 2) AND (co != 0)\n)\nWHERE (co != 0) AND (co2 != 2) 1 0 3 1 1 0 0 1 SELECT alias AS name\nFROM \n(\n SELECT name AS alias\n FROM system.settings\n WHERE alias = \'enable_optimize_predicate_expression\'\n)\nANY INNER JOIN \n(\n SELECT name\n FROM system.settings\n) USING (name)\nWHERE name = \'enable_optimize_predicate_expression\' From 88d08f9f39854d3f4e12192f11426b251588a280 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 2 May 2020 13:34:45 +0300 Subject: [PATCH 011/150] Move tests from 00740_optimize_predicate_expression to 01056_predicate_optimizer_bugs --- .../00740_optimize_predicate_expression.reference | 1 - .../00740_optimize_predicate_expression.sql | 11 ----------- .../01056_predicate_optimizer_bugs.reference | 1 + .../0_stateless/01056_predicate_optimizer_bugs.sql | 10 ++++++++++ 4 files changed, 11 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/00740_optimize_predicate_expression.reference b/tests/queries/0_stateless/00740_optimize_predicate_expression.reference index 6ec592953a9..6db331af725 100644 --- a/tests/queries/0_stateless/00740_optimize_predicate_expression.reference +++ b/tests/queries/0_stateless/00740_optimize_predicate_expression.reference @@ -1,2 +1 @@ nan -SELECT dummy\nFROM system.one\nWHERE (dummy > 0) AND (dummy < 0) diff --git a/tests/queries/0_stateless/00740_optimize_predicate_expression.sql b/tests/queries/0_stateless/00740_optimize_predicate_expression.sql index 78065c9896e..b016ab49ddd 100644 --- a/tests/queries/0_stateless/00740_optimize_predicate_expression.sql +++ b/tests/queries/0_stateless/00740_optimize_predicate_expression.sql @@ -32,15 +32,4 @@ FROM ( WHERE user_id = 999 ) js2 USING site); --- check order is preserved -SET enable_debug_queries = 1; -ANALYZE SELECT * FROM system.one HAVING dummy > 0 AND dummy < 0; - --- from #10613 -SELECT name, count() AS cnt -FROM remote('127.{1,2}', system.settings) -GROUP BY name -HAVING (max(value) > '9') AND (min(changed) = 0) -FORMAT Null; - DROP TABLE perf; diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index 3fbdd87eec4..bd132202979 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -26,3 +26,4 @@ SELECT dummy\nFROM \n(\n SELECT dummy\n FROM system.one\n WHERE arrayMa 0 SELECT \n id, \n value, \n value_1\nFROM \n(\n SELECT \n 1 AS id, \n 2 AS value\n)\nALL INNER JOIN \n(\n SELECT \n 1 AS id, \n 3 AS value_1\n) USING (id)\nWHERE arrayMap(x -> ((x + value) + value_1), [1]) = [6] 1 2 3 +SELECT dummy\nFROM system.one\nWHERE (dummy > 0) AND (dummy < 0) diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql index db6c78e3cc6..18552a6591d 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.sql @@ -74,3 +74,13 @@ SELECT * FROM (SELECT * FROM system.one) WHERE arrayMap(x -> x + 1, [dummy]) = [ ANALYZE SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6]; SELECT * FROM (SELECT 1 AS id, 2 AS value) INNER JOIN (SELECT 1 AS id, 3 AS value_1) USING id WHERE arrayMap(x -> x + value + value_1, [1]) = [6]; + +-- check order is preserved +ANALYZE SELECT * FROM system.one HAVING dummy > 0 AND dummy < 0; + +-- from #10613 +SELECT name, count() AS cnt +FROM remote('127.{1,2}', system.settings) +GROUP BY name +HAVING (max(value) > '9') AND (min(changed) = 0) +FORMAT Null; From 1f45c4302a4d10770e541eb768f65c15de9461ff Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 2 May 2020 14:45:01 +0300 Subject: [PATCH 012/150] Drop check for getPositionByName() (it already has it) in ExpressionActions --- src/Interpreters/ExpressionActions.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 363d4765019..c86239b71ae 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -342,11 +342,7 @@ void ExpressionAction::execute(Block & block, bool dry_run, ExtraBlockPtr & not_ { ColumnNumbers arguments(argument_names.size()); for (size_t i = 0; i < argument_names.size(); ++i) - { - if (!block.has(argument_names[i])) - throw Exception("Not found column: '" + argument_names[i] + "'", ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK); arguments[i] = block.getPositionByName(argument_names[i]); - } size_t num_columns_without_result = block.columns(); block.insert({ nullptr, result_type, result_name}); From 6a9ec1396ae0b4c46aecafa4cb83fd895653d231 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 2 May 2020 14:27:25 +0300 Subject: [PATCH 013/150] Access aggregate keys by names over indexes Relax this to allow accepting aggregate keys in different order (typical use case is distributed queries, where the initiator server will do final merge). --- src/Interpreters/Aggregator.cpp | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 7cd6a8341cd..54f54b9b69e 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -127,11 +127,11 @@ Block Aggregator::getHeader(bool final) const if (final) { - for (size_t i = 0; i < params.aggregates_size; ++i) + for (const auto & aggregate : params.aggregates) { - auto & elem = res.getByPosition(params.keys_size + i); + auto & elem = res.getByName(aggregate.column_name); - elem.type = params.aggregates[i].function->getReturnType(); + elem.type = aggregate.function->getReturnType(); elem.column = elem.type->createColumn(); } } @@ -1057,7 +1057,8 @@ Block Aggregator::prepareBlockAndFill( { if (!final) { - aggregate_columns[i] = header.safeGetByPosition(i + params.keys_size).type->createColumn(); + const auto & aggregate_column_name = params.aggregates[i].column_name; + aggregate_columns[i] = header.getByName(aggregate_column_name).type->createColumn(); /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); @@ -1093,10 +1094,11 @@ Block Aggregator::prepareBlockAndFill( for (size_t i = 0; i < params.aggregates_size; ++i) { + const auto & aggregate_column_name = params.aggregates[i].column_name; if (final) - res.getByPosition(i + params.keys_size).column = std::move(final_aggregate_columns[i]); + res.getByName(aggregate_column_name).column = std::move(final_aggregate_columns[i]); else - res.getByPosition(i + params.keys_size).column = std::move(aggregate_columns[i]); + res.getByName(aggregate_column_name).column = std::move(aggregate_columns[i]); } /// Change the size of the columns-constants in the block. @@ -1821,7 +1823,10 @@ void NO_INLINE Aggregator::mergeStreamsImplCase( key_columns[i] = block.safeGetByPosition(i).column.get(); for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i] = &typeid_cast(*block.safeGetByPosition(params.keys_size + i).column).getData(); + { + const auto & aggregate_column_name = params.aggregates[i].column_name; + aggregate_columns[i] = &typeid_cast(*block.getByName(aggregate_column_name).column).getData(); + } typename Method::State state(key_columns, key_sizes, aggregation_state_cache); @@ -1897,7 +1902,10 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl( /// Remember the columns we will work with for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i] = &typeid_cast(*block.safeGetByPosition(params.keys_size + i).column).getData(); + { + const auto & aggregate_column_name = params.aggregates[i].column_name; + aggregate_columns[i] = &typeid_cast(*block.getByName(aggregate_column_name).column).getData(); + } AggregatedDataWithoutKey & res = result.without_key; if (!res) From b781e08e9a10b999f5a6bf6402c696f84089774d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sun, 24 May 2020 01:23:26 +0300 Subject: [PATCH 014/150] draft --- CHANGELOG.md | 100 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 100 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index ca14134753a..61ca07f9bb9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,50 @@ ## ClickHouse release v20.4 +### ClickHouse release v20.4.3.16-stable 2020-05-23 + +#### Bug Fix + +* Remove logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). +* Fixed memory leak in registerDiskS3. [#11074](https://github.com/ClickHouse/ClickHouse/pull/11074) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fixes the potential missed data during termination of Kafka engine table. [#11048](https://github.com/ClickHouse/ClickHouse/pull/11048) ([filimonov](https://github.com/filimonov)). +* Fixed parseDateTime64BestEffort argument resolution bugs. ... [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). +* Fix very rare potential use-after-free error in MergeTree if table was not created successfully. [#10986](https://github.com/ClickHouse/ClickHouse/pull/10986) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix metadata (relative path for rename) and data (relative path for symlink) handling for Atomic database. [#10980](https://github.com/ClickHouse/ClickHouse/pull/10980) ([Azat Khuzhin](https://github.com/azat)). +* Fix very rare potential use-after-free error in MergeTree if table was not created successfully. [#10970](https://github.com/ClickHouse/ClickHouse/pull/10970) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix server crash on concurrent `ALTER` and `DROP DATABASE` queries with `Atomic` database engine. [#10968](https://github.com/ClickHouse/ClickHouse/pull/10968) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect raw data size in method getRawData(). [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). +* Fix incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Avoid sending partially written files by the DistributedBlockOutputStream. [#10940](https://github.com/ClickHouse/ClickHouse/pull/10940) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash in `SELECT count(notNullIn(NULL, []))`. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix for the hang which was happening sometimes during DROP of table engine=Kafka (or during server restarts). [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). +* Now it's possible to execute multiple `ALTER RENAME` like `a TO b, c TO a`. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). +* Fix possible race which could happen when you get result from aggregate function state from multiple thread for the same column. The only way (which I found) it can happen is when you use `finalizeAggregation` function while reading from table with `Memory` engine which stores `AggregateFunction` state for `quanite*` function. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). +* Fix SIGSEGV in StringHashTable (if such key does not exist). [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `WATCH` hangs after `LiveView` table was dropped from database with `Atomic` engine. [#10859](https://github.com/ClickHouse/ClickHouse/pull/10859) ([tavplubix](https://github.com/tavplubix)). +* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). +* Now constraints are updated if the column participating in `CONSTRAINT` expression was renamed. Fixes [#10844](https://github.com/ClickHouse/ClickHouse/issues/10844). [#10847](https://github.com/ClickHouse/ClickHouse/pull/10847) ([alesapin](https://github.com/alesapin)). +* Fix potential read of uninitialized memory in cache dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix columns order after Block::sortColumns() (also add a test that shows that it affects some real use case - Buffer engine). [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). +* Fix the issue with ODBC bridge when no quoting of identifiers is requested. This fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix UBSan and MSan report in DateLUT. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* -. [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). +* Fix `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix combinator -OrNull and -OrDefault when combined with -State. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). +* Fix possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). +* Fix nullptr dereference in StorageBuffer if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix optimize_skip_unused_shards with LowCardinality. [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). +* Fixed handling condition variable for synchronous mutations. In some cases signals to that condition variable could be lost. [#10588](https://github.com/ClickHouse/ClickHouse/pull/10588) ([Vladimir Chebotarev](https://github.com/excitoon)). +* This PR fixes possible crash when `createDictionary()` is called before `loadStoredObject()` has finished. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix SELECT of column ALIAS which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). +* * implemented comparison between DateTime64 and String values (just like for DateTime). ... [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). +* Disable GROUP BY sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for WITH ROLLUP/CUBE/TOTALS. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). +* Fixes: [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263) (after that PR dist send via INSERT had been postponing on each INSERT) Fixes: [#8756](https://github.com/ClickHouse/ClickHouse/issues/8756) (that PR breaks distributed sends with all of the following conditions met (unlikely setup for now I guess): `internal_replication == false`, multiple local shards (activates the hardlinking code) and `distributed_storage_policy` (makes `link(2)` fails on `EXDEV`)). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). +* Todo. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +* #10551. [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). + ### ClickHouse release v20.4.2.9, 2020-05-12 #### Backward Incompatible Change @@ -280,6 +325,61 @@ ## ClickHouse release v20.3 +### ClickHouse release v20.3.10.75-lts 2020-05-23 + +#### Bug Fix + +* Remove logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). +* Fixed parseDateTime64BestEffort argument resolution bugs. ... [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). +* Fix incorrect raw data size in method getRawData(). [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). +* Fix incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). +* Fix SIGSEGV in StringHashTable (if such key does not exist). [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). +* Fix columns order after Block::sortColumns() (also add a test that shows that it affects some real use case - Buffer engine). [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). +* Fix the issue with ODBC bridge when no quoting of identifiers is requested. This fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix UBSan and MSan report in DateLUT. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* -. [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). +* Fix `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix combinator -OrNull and -OrDefault when combined with -State. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). +* Fix crash in `generateRandom` with nested types. Fixes [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix disappearing totals. Totals could have being filtered if query had had join or subquery with external where condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix multiple usages of `IN` operator with the identical set in one query. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). +* Fix order of parameters in AggregateTransform constructor. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). +* Fix the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible incorrect number of rows for queries with `LIMIT`. Fixes [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709). [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). +* On `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). +* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix SELECT of column ALIAS which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). +* * implemented comparison between DateTime64 and String values (just like for DateTime). ... [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). +* Fix index corruption, which may accur in some cases after merge compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)). +* Now the situation, when mutation finished all parts, but hung up in `is_done=0`, is impossible. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). +* Fix overflow at beginning of unix epoch for timezones with fractional offset from UTC. This fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Proper Distributed shutdown (fixes UAF, avoid waiting for sending all batches). [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). +* add CA certificates to clickhouse-server docker image. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). +* Fix numeric overflow in simpleLinearRegression over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). +* Minor improvements in printing diagnostic info in input formats. Fixes [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204). [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)). +* Todo. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement + +* Fix UBSan report in LZ4 library. [#10631](https://github.com/ClickHouse/ClickHouse/pull/10631) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix clang-10 build. https://github.com/ClickHouse/ClickHouse/issues/10238. [#10370](https://github.com/ClickHouse/ClickHouse/pull/10370) ([Amos Bird](https://github.com/amosbird)). + +#### NO CL ENTRY + +* NO CL ENTRY: '20.3 backports part 2'. [#11157](https://github.com/ClickHouse/ClickHouse/pull/11157) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* NO CL ENTRY: '20.3 backports part 1'. [#11018](https://github.com/ClickHouse/ClickHouse/pull/11018) ([Ivan](https://github.com/abyss7)). + +#### Bug fix + +* #10551. [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). + + ### ClickHouse release v20.3.8.53, 2020-04-23 #### Bug Fix From 24271ec50faa317b3604b49d90767f74f81bd14e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 22 May 2020 18:46:31 +0300 Subject: [PATCH 015/150] Test for data skip index over column under aggregation Affected versions: 19.17.9.60+ --- ...data_skip_index_over_aggregation.reference | 20 +++++++++++ ...01285_data_skip_index_over_aggregation.sql | 34 +++++++++++++++++++ 2 files changed, 54 insertions(+) create mode 100644 tests/queries/0_stateless/01285_data_skip_index_over_aggregation.reference create mode 100644 tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql diff --git a/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.reference b/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.reference new file mode 100644 index 00000000000..0d221d16995 --- /dev/null +++ b/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.reference @@ -0,0 +1,20 @@ +INSERT +1 0 +1 1 +1 1 +INSERT +1 0 +1 1 +1 0 +1 1 +1 2 +1 3 +1 1 +1 1 +1 3 +OPTIMIZE +1 3 +1 3 +OPTIMIZE +1 3 +1 3 diff --git a/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql b/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql new file mode 100644 index 00000000000..110c5b65cab --- /dev/null +++ b/tests/queries/0_stateless/01285_data_skip_index_over_aggregation.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS data_01285; + +SET max_threads=1; + + +CREATE TABLE data_01285 ( + key Int, + value SimpleAggregateFunction(max, Nullable(Int)), + INDEX value_idx assumeNotNull(value) TYPE minmax GRANULARITY 1 +) +ENGINE=AggregatingMergeTree() +ORDER BY key; + +SELECT 'INSERT'; +INSERT INTO data_01285 SELECT 1, number FROM numbers(2); +SELECT * FROM data_01285; +SELECT * FROM data_01285 WHERE assumeNotNull(value) = 1; +SELECT 'INSERT'; +INSERT INTO data_01285 SELECT 1, number FROM numbers(4); +SELECT * FROM data_01285; +SELECT * FROM data_01285 WHERE assumeNotNull(value) = 1; +SELECT * FROM data_01285 WHERE assumeNotNull(value) = 3; +SELECT 'OPTIMIZE'; +OPTIMIZE TABLE data_01285 FINAL; +SELECT * FROM data_01285; +-- before the fix value_idx contains one range {0, 0} +-- and hence cannot find these record. +SELECT * FROM data_01285 WHERE assumeNotNull(value) = 3; +-- one more time just in case +SELECT 'OPTIMIZE'; +OPTIMIZE TABLE data_01285 FINAL; +SELECT * FROM data_01285; +-- and this passes even without fix. +SELECT * FROM data_01285 WHERE assumeNotNull(value) = 3; From 71731b2e9975c76819193e95a838e5bb33e26d06 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 02:02:27 +0300 Subject: [PATCH 016/150] Fix data skipping indexes for columns with additional actions during merge For example for AggregatingMergeTree if the column in the data skipping index is under some aggregate function, then data skipping index will be incorrect. Fix this by calculating data skipping indexes after aggregation has been done. --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 00b474a7792..b3833255ee9 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -714,16 +714,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor MergeProgressCallback(merge_entry, watch_prev_elapsed, horizontal_stage_progress)); Pipe pipe(std::move(input)); - - if (data.hasPrimaryKey() || data.hasSkipIndices()) - { - auto expr = std::make_shared(pipe.getHeader(), data.sorting_key_and_skip_indices_expr); - pipe.addSimpleTransform(std::move(expr)); - - auto materializing = std::make_shared(pipe.getHeader()); - pipe.addSimpleTransform(std::move(materializing)); - } - pipes.emplace_back(std::move(pipe)); } @@ -796,6 +786,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (need_remove_expired_values) merged_stream = std::make_shared(merged_stream, data, new_data_part, time_of_merge, force_ttl); + if (data.hasPrimaryKey() || data.hasSkipIndices()) + { + merged_stream = std::make_shared(merged_stream, data.sorting_key_and_skip_indices_expr); + merged_stream = std::make_shared(merged_stream); + } + + MergedBlockOutputStream to{ new_data_part, merging_columns, From c36708caf9885295a25d5c7f47c8271bba63a855 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 24 May 2020 16:06:52 +0300 Subject: [PATCH 017/150] Split data skipping indexes from the order by to evaluate order by before merge --- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++++++ src/Storages/MergeTree/MergeTreeData.h | 1 + src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 11 +++++++++-- 3 files changed, 18 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 88a396ce763..5284705de5c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -419,6 +419,7 @@ void MergeTreeData::setProperties(const StorageInMemoryMetadata & metadata, bool } ASTPtr skip_indices_with_primary_key_expr_list = new_primary_key_expr_list->clone(); + ASTPtr skip_indices_expr_list = new_primary_key_expr_list->clone(); ASTPtr skip_indices_with_sorting_key_expr_list = new_sorting_key_expr_list->clone(); MergeTreeIndices new_indices; @@ -448,6 +449,7 @@ void MergeTreeData::setProperties(const StorageInMemoryMetadata & metadata, bool { skip_indices_with_primary_key_expr_list->children.push_back(expr->clone()); skip_indices_with_sorting_key_expr_list->children.push_back(expr->clone()); + skip_indices_expr_list->children.push_back(expr->clone()); } indices_names.insert(new_indices.back()->name); @@ -458,6 +460,11 @@ void MergeTreeData::setProperties(const StorageInMemoryMetadata & metadata, bool auto new_indices_with_primary_key_expr = ExpressionAnalyzer( skip_indices_with_primary_key_expr_list, syntax_primary, global_context).getActions(false); + auto syntax_indices = SyntaxAnalyzer(global_context).analyze( + skip_indices_with_primary_key_expr_list, all_columns); + auto new_indices_expr = ExpressionAnalyzer( + skip_indices_expr_list, syntax_indices, global_context).getActions(false); + auto syntax_sorting = SyntaxAnalyzer(global_context).analyze( skip_indices_with_sorting_key_expr_list, all_columns); auto new_indices_with_sorting_key_expr = ExpressionAnalyzer( @@ -484,6 +491,7 @@ void MergeTreeData::setProperties(const StorageInMemoryMetadata & metadata, bool setConstraints(metadata.constraints); + skip_indices_expr = new_indices_expr; primary_key_and_skip_indices_expr = new_indices_with_primary_key_expr; sorting_key_and_skip_indices_expr = new_indices_with_sorting_key_expr; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 9cf72cbe8bb..89aa3423778 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -661,6 +661,7 @@ public: /// Secondary (data skipping) indices for MergeTree MergeTreeIndices skip_indices; + ExpressionActionsPtr skip_indices_expr; ExpressionActionsPtr primary_key_and_skip_indices_expr; ExpressionActionsPtr sorting_key_and_skip_indices_expr; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index b3833255ee9..cf187310ae0 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -714,6 +714,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor MergeProgressCallback(merge_entry, watch_prev_elapsed, horizontal_stage_progress)); Pipe pipe(std::move(input)); + + if (data.hasSortingKey()) + { + auto expr = std::make_shared(pipe.getHeader(), data.sorting_key_expr); + pipe.addSimpleTransform(std::move(expr)); + } + pipes.emplace_back(std::move(pipe)); } @@ -786,9 +793,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (need_remove_expired_values) merged_stream = std::make_shared(merged_stream, data, new_data_part, time_of_merge, force_ttl); - if (data.hasPrimaryKey() || data.hasSkipIndices()) + if (data.hasSkipIndices()) { - merged_stream = std::make_shared(merged_stream, data.sorting_key_and_skip_indices_expr); + merged_stream = std::make_shared(merged_stream, data.skip_indices_expr); merged_stream = std::make_shared(merged_stream); } From 9b3cc9e5252d6c3905338d1951b12720daffccdc Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 May 2020 20:07:14 +0300 Subject: [PATCH 018/150] Almost working ttl in IStorage --- src/DataStreams/TTLBlockInputStream.cpp | 11 +- src/Storages/IStorage.cpp | 46 +++++++ src/Storages/IStorage.h | 24 +++- src/Storages/MergeTree/MergeTreeData.cpp | 124 +++++++----------- src/Storages/MergeTree/MergeTreeData.h | 46 ++----- .../MergeTree/MergeTreeDataWriter.cpp | 13 +- .../MergeTree/MergeTreePartsMover.cpp | 8 +- .../ReplicatedMergeTreeTableMetadata.cpp | 2 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 5 +- src/Storages/StorageInMemoryMetadata.cpp | 70 ++++++++++ src/Storages/StorageInMemoryMetadata.h | 14 ++ 11 files changed, 228 insertions(+), 135 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 692d2c0d5cf..5064e2236f8 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -110,10 +110,11 @@ void TTLBlockInputStream::readSuffixImpl() void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) { - storage.rows_ttl_entry.expression->execute(block); + const auto & rows_ttl = storage.getRowsTTL(); + rows_ttl.expression->execute(block); const IColumn * ttl_column = - block.getByName(storage.rows_ttl_entry.result_column).column.get(); + block.getByName(rows_ttl.result_column).column.get(); const auto & column_names = header.getNames(); MutableColumns result_columns; @@ -152,7 +153,8 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) } std::vector columns_to_remove; - for (const auto & [name, ttl_entry] : storage.column_ttl_entries_by_name) + const auto & column_ttl_entries_by_name = storage.getColumnTTLs(); + for (const auto & [name, ttl_entry] : column_ttl_entries_by_name) { /// If we read not all table columns. E.g. while mutation. if (!block.has(name)) @@ -212,8 +214,9 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) void TTLBlockInputStream::updateMovesTTL(Block & block) { + const auto & move_ttl_entries = storage.getMoveTTLs(); std::vector columns_to_remove; - for (const auto & ttl_entry : storage.move_ttl_entries) + for (const auto & ttl_entry : move_ttl_entries) { auto & new_ttl_info = new_ttl_infos.moves_ttl[ttl_entry.result_column]; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 64e06a2534e..f8f2599184f 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -541,4 +541,50 @@ Names IStorage::getColumnsRequiredForSampling() const return {}; } + +const StorageMetadataTTLField & IStorage::getRowsTTL() const +{ + return table_ttl.rows_ttl; +} + +bool IStorage::hasRowsTTL() const +{ + return table_ttl.rows_ttl.expression != nullptr; +} + +const StorageMetadataTTLFields & IStorage::getMoveTTLs() const +{ + return table_ttl.move_ttl; +} + +bool IStorage::hasAnyMoveTTL() const +{ + return !table_ttl.move_ttl.empty(); +} + +const StorageMetadataTTLColumnEntries & IStorage::getColumnTTLs() const +{ + return column_ttls_by_name; +} + +void IStorage::setColumnTTLs(const StorageMetadataTTLColumnEntries & column_ttls_by_name_) +{ + column_ttls_by_name = column_ttls_by_name_; +} + +const StorageMetadataTableTTL & IStorage::getTableTTLs() const +{ + return table_ttl; +} + +void IStorage::setTableTTLs(const StorageMetadataTableTTL & table_ttl_) +{ + table_ttl = table_ttl_; +} + +bool IStorage::hasAnyTableTTL() const +{ + return hasAnyMoveTTL() || hasRowsTTL(); +} + } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index f707b3a4b5c..416d8ffbbb1 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -130,10 +130,7 @@ public: virtual bool hasEvenlyDistributedRead() const { return false; } /// Returns true if there is set table TTL, any column TTL or any move TTL. - virtual bool hasAnyTTL() const { return false; } - - /// Returns true if there is set TTL for rows. - virtual bool hasRowsTTL() const { return false; } + virtual bool hasAnyTTL() const { return hasRowsTTL() || hasAnyTableTTL(); } /// Optional size information of each physical column. /// Currently it's only used by the MergeTree family for query optimizations. @@ -205,6 +202,9 @@ private: StorageMetadataKeyField sorting_key; StorageMetadataKeyField sampling_key; + StorageMetadataTTLColumnEntries column_ttls_by_name; + StorageMetadataTableTTL table_ttl; + private: RWLockImpl::LockHolder tryLockTimed( const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const SettingSeconds & acquire_timeout) const; @@ -514,6 +514,22 @@ public: /// Returns storage policy if storage supports it virtual StoragePolicyPtr getStoragePolicy() const { return {}; } + /// Returns true if there is set TTL for rows. + const StorageMetadataTTLField & getRowsTTL() const; + bool hasRowsTTL() const; + + const StorageMetadataTTLFields & getMoveTTLs() const; + bool hasAnyMoveTTL() const; + + const StorageMetadataTableTTL & getTableTTLs() const; + void setTableTTLs(const StorageMetadataTableTTL & table_ttl_); + bool hasAnyTableTTL() const; + + const StorageMetadataTTLColumnEntries & getColumnTTLs() const; + void setColumnTTLs(const StorageMetadataTTLColumnEntries & column_ttls_by_name_); + bool hasAnyColumnTTL() const { return !column_ttls_by_name.empty(); } + + /// If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it. /// Used for: /// - Simple count() opimization diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 142d503569f..a9ba8c871b0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -580,55 +580,17 @@ void MergeTreeData::initPartitionKey(ASTPtr partition_by_ast) setPartitionKey(new_partition_key); } -namespace -{ - -void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name) -{ - for (const auto & action : ttl_expression->getActions()) - { - if (action.type == ExpressionAction::APPLY_FUNCTION) - { - IFunctionBase & func = *action.function_base; - if (!func.isDeterministic()) - throw Exception("TTL expression cannot contain non-deterministic functions, " - "but contains function " + func.getName(), ErrorCodes::BAD_ARGUMENTS); - } - } - - const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name); - - if (!typeid_cast(result_column.type.get()) - && !typeid_cast(result_column.type.get())) - { - throw Exception("TTL expression result column should have DateTime or Date type, but has " - + result_column.type->getName(), ErrorCodes::BAD_TTL_EXPRESSION); - } -} - -} void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, const ASTPtr & new_ttl_table_ast, bool only_check) { - auto new_column_ttls = new_columns.getColumnTTLs(); + auto new_column_ttl_asts = new_columns.getColumnTTLs(); - auto create_ttl_entry = [this, &new_columns](ASTPtr ttl_ast) - { - TTLEntry result; + StorageMetadataTTLColumnEntries new_column_ttl_by_name = getColumnTTLs(); - auto syntax_result = SyntaxAnalyzer(global_context).analyze(ttl_ast, new_columns.getAllPhysical()); - result.expression = ExpressionAnalyzer(ttl_ast, syntax_result, global_context).getActions(false); - result.destination_type = DataDestinationType::DELETE; - result.result_column = ttl_ast->getColumnName(); - - checkTTLExpression(result.expression, result.result_column); - return result; - }; - - if (!new_column_ttls.empty()) + if (!new_column_ttl_asts.empty()) { NameSet columns_ttl_forbidden; @@ -640,23 +602,24 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, for (const auto & col : getColumnsRequiredForSortingKey()) columns_ttl_forbidden.insert(col); - for (const auto & [name, ast] : new_column_ttls) + for (const auto & [name, ast] : new_column_ttl_asts) { if (columns_ttl_forbidden.count(name)) throw Exception("Trying to set TTL for key column " + name, ErrorCodes::ILLEGAL_COLUMN); else { - auto new_ttl_entry = create_ttl_entry(ast); - if (!only_check) - column_ttl_entries_by_name[name] = new_ttl_entry; + auto new_ttl_entry = StorageMetadataTTLField::getTTLFromAST(ast, new_columns, global_context); + new_column_ttl_by_name[name] = new_ttl_entry; } } + if (!only_check) + setColumnTTLs(new_column_ttl_by_name); } if (new_ttl_table_ast) { - std::vector update_move_ttl_entries; - TTLEntry update_rows_ttl_entry; + StorageMetadataTTLFields update_move_ttl_entries; + StorageMetadataTTLField update_rows_ttl_entry; bool seen_delete_ttl = false; for (const auto & ttl_element_ptr : new_ttl_table_ast->children) @@ -672,20 +635,16 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, throw Exception("More than one DELETE TTL expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION); } - auto new_rows_ttl_entry = create_ttl_entry(ttl_element->children[0]); - if (!only_check) - update_rows_ttl_entry = new_rows_ttl_entry; + auto new_rows_ttl_entry = StorageMetadataTTLField::getTTLFromAST(ttl_element_ptr, new_columns, global_context); + update_rows_ttl_entry = new_rows_ttl_entry; seen_delete_ttl = true; } else { - auto new_ttl_entry = create_ttl_entry(ttl_element->children[0]); + auto new_ttl_entry = StorageMetadataTTLField::getTTLFromAST(ttl_element_ptr, new_columns, global_context); - new_ttl_entry.entry_ast = ttl_element_ptr; - new_ttl_entry.destination_type = ttl_element->destination_type; - new_ttl_entry.destination_name = ttl_element->destination_name; - if (!new_ttl_entry.getDestination(getStoragePolicy())) + if (!getDestinationForTTL(new_ttl_entry)) { String message; if (new_ttl_entry.destination_type == DataDestinationType::DISK) @@ -695,18 +654,21 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, throw Exception(message, ErrorCodes::BAD_TTL_EXPRESSION); } - if (!only_check) - update_move_ttl_entries.emplace_back(std::move(new_ttl_entry)); + update_move_ttl_entries.emplace_back(std::move(new_ttl_entry)); } } if (!only_check) { - rows_ttl_entry = update_rows_ttl_entry; - ttl_table_ast = new_ttl_table_ast; + StorageMetadataTableTTL new_table_ttl + { + .definition_ast = new_ttl_table_ast, + .rows_ttl = update_rows_ttl_entry, + .move_ttl = update_move_ttl_entries, + }; auto move_ttl_entries_lock = std::lock_guard(move_ttl_entries_mutex); - move_ttl_entries = update_move_ttl_entries; + setTableTTLs(new_table_ttl); } } } @@ -2918,7 +2880,7 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ auto ttl_entry = selectTTLEntryForTTLInfos(ttl_infos, time_of_move); if (ttl_entry) { - SpacePtr destination_ptr = ttl_entry->getDestination(getStoragePolicy()); + SpacePtr destination_ptr = getDestinationForTTL(*ttl_entry); if (!destination_ptr) { if (ttl_entry->destination_type == DataDestinationType::VOLUME) @@ -2952,37 +2914,39 @@ ReservationPtr MergeTreeData::tryReserveSpacePreferringTTLRules(UInt64 expected_ return reservation; } -SpacePtr MergeTreeData::TTLEntry::getDestination(StoragePolicyPtr policy) const +SpacePtr MergeTreeData::getDestinationForTTL(const StorageMetadataTTLField & ttl) const { - if (destination_type == DataDestinationType::VOLUME) - return policy->getVolumeByName(destination_name); - else if (destination_type == DataDestinationType::DISK) - return policy->getDiskByName(destination_name); + auto policy = getStoragePolicy(); + if (ttl.destination_type == DataDestinationType::VOLUME) + return policy->getVolumeByName(ttl.destination_name); + else if (ttl.destination_type == DataDestinationType::DISK) + return policy->getDiskByName(ttl.destination_name); else return {}; } -bool MergeTreeData::TTLEntry::isPartInDestination(StoragePolicyPtr policy, const IMergeTreeDataPart & part) const +bool MergeTreeData::isPartInTTLDestination(const StorageMetadataTTLField & ttl, const IMergeTreeDataPart & part) const { - if (destination_type == DataDestinationType::VOLUME) + auto policy = getStoragePolicy(); + if (ttl.destination_type == DataDestinationType::VOLUME) { - for (const auto & disk : policy->getVolumeByName(destination_name)->getDisks()) + for (const auto & disk : policy->getVolumeByName(ttl.destination_name)->getDisks()) if (disk->getName() == part.volume->getDisk()->getName()) return true; } - else if (destination_type == DataDestinationType::DISK) - return policy->getDiskByName(destination_name)->getName() == part.volume->getDisk()->getName(); + else if (ttl.destination_type == DataDestinationType::DISK) + return policy->getDiskByName(ttl.destination_name)->getName() == part.volume->getDisk()->getName(); return false; } -std::optional MergeTreeData::selectTTLEntryForTTLInfos( - const IMergeTreeDataPart::TTLInfos & ttl_infos, - time_t time_of_move) const +std::optional +MergeTreeData::selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const { time_t max_max_ttl = 0; - std::vector::const_iterator best_entry_it; + std::vector::const_iterator best_entry_it; auto lock = std::lock_guard(move_ttl_entries_mutex); + const auto & move_ttl_entries = getMoveTTLs(); for (auto ttl_entry_it = move_ttl_entries.begin(); ttl_entry_it != move_ttl_entries.end(); ++ttl_entry_it) { auto ttl_info_it = ttl_infos.moves_ttl.find(ttl_entry_it->result_column); @@ -2996,7 +2960,7 @@ std::optional MergeTreeData::selectTTLEntryForTTLInfos( } } - return max_max_ttl ? *best_entry_it : std::optional(); + return max_max_ttl ? *best_entry_it : std::optional(); } MergeTreeData::DataParts MergeTreeData::getDataParts(const DataPartStates & affordable_states) const @@ -3416,7 +3380,7 @@ bool MergeTreeData::areBackgroundMovesNeeded() const if (policy->getVolumes().size() > 1) return true; - return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->getDisks().size() > 1 && !move_ttl_entries.empty(); + return policy->getVolumes().size() == 1 && policy->getVolumes()[0]->getDisks().size() > 1 && hasAnyMoveTTL(); } bool MergeTreeData::movePartsToSpace(const DataPartsVector & parts, SpacePtr space) @@ -3555,7 +3519,7 @@ ColumnDependencies MergeTreeData::getColumnDependencies(const NameSet & updated_ if (hasRowsTTL()) { - if (add_dependent_columns(rows_ttl_entry.expression, required_ttl_columns)) + if (add_dependent_columns(getRowsTTL().expression, required_ttl_columns)) { /// Filter all columns, if rows TTL expression have to be recalculated. for (const auto & column : getColumns().getAllPhysical()) @@ -3563,13 +3527,13 @@ ColumnDependencies MergeTreeData::getColumnDependencies(const NameSet & updated_ } } - for (const auto & [name, entry] : column_ttl_entries_by_name) + for (const auto & [name, entry] : getColumnTTLs()) { if (add_dependent_columns(entry.expression, required_ttl_columns)) updated_ttl_columns.insert(name); } - for (const auto & entry : move_ttl_entries) + for (const auto & entry : getMoveTTLs()) add_dependent_columns(entry.expression, required_ttl_columns); for (const auto & column : indices_columns) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2b5b75110f1..b1e14188e21 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -518,11 +518,6 @@ public: bool hasSkipIndices() const { return !skip_indices.empty(); } - bool hasAnyColumnTTL() const { return !column_ttl_entries_by_name.empty(); } - bool hasAnyMoveTTL() const { return !move_ttl_entries.empty(); } - bool hasRowsTTL() const override { return !rows_ttl_entry.isEmpty(); } - bool hasAnyTTL() const override { return hasRowsTTL() || hasAnyMoveTTL() || hasAnyColumnTTL(); } - /// Check that the part is not broken and calculate the checksums for it if they are not present. MutableDataPartPtr loadPartAndFixMetadata(const VolumePtr & volume, const String & relative_path) const; @@ -624,6 +619,13 @@ public: /// Return alter conversions for part which must be applied on fly. AlterConversions getAlterConversionsForPart(const MergeTreeDataPartPtr part) const; + /// Returns destination disk or volume for the TTL rule according to current + /// storage policy + SpacePtr getDestinationForTTL(const StorageMetadataTTLField & ttl) const; + + /// Checks if given part already belongs destination disk or volume for the + /// TTL rule. + bool isPartInTTLDestination(const StorageMetadataTTLField & ttl, const IMergeTreeDataPart & part) const; MergeTreeDataFormatVersion format_version; @@ -646,39 +648,13 @@ public: ExpressionActionsPtr primary_key_and_skip_indices_expr; ExpressionActionsPtr sorting_key_and_skip_indices_expr; - struct TTLEntry - { - ExpressionActionsPtr expression; - String result_column; + std::optional selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const; - /// Name and type of a destination are only valid in table-level context. - DataDestinationType destination_type; - String destination_name; - - ASTPtr entry_ast; - - /// Returns destination disk or volume for this rule. - SpacePtr getDestination(StoragePolicyPtr policy) const; - - /// Checks if given part already belongs destination disk or volume for this rule. - bool isPartInDestination(StoragePolicyPtr policy, const IMergeTreeDataPart & part) const; - - bool isEmpty() const { return expression == nullptr; } - }; - - std::optional selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const; - - using TTLEntriesByName = std::unordered_map; - TTLEntriesByName column_ttl_entries_by_name; - - TTLEntry rows_ttl_entry; - - /// This mutex is required for background move operations which do not obtain global locks. + /// This mutex is required for background move operations which do not + /// obtain global locks. + /// TODO (alesap) It will be removed after metadata became atomic mutable std::mutex move_ttl_entries_mutex; - /// Vector rw operations have to be done under "move_ttl_entries_mutex". - std::vector move_ttl_entries; - /// Limiting parallel sends per one table, used in DataPartsExchange std::atomic_uint current_table_sends {0}; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 48dbf66e72b..0209e6cc23d 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -77,10 +77,12 @@ void buildScatterSelector( } /// Computes ttls and updates ttl infos -void updateTTL(const MergeTreeData::TTLEntry & ttl_entry, +void updateTTL( + const StorageMetadataTTLField & ttl_entry, IMergeTreeDataPart::TTLInfos & ttl_infos, DB::MergeTreeDataPartTTLInfo & ttl_info, - Block & block, bool update_part_min_max_ttls) + Block & block, + bool update_part_min_max_ttls) { bool remove_column = false; if (!block.has(ttl_entry.result_column)) @@ -228,7 +230,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa size_t expected_size = block.bytes(); DB::IMergeTreeDataPart::TTLInfos move_ttl_infos; - for (const auto & ttl_entry : data.move_ttl_entries) + const auto & move_ttl_entries = data.getMoveTTLs(); + for (const auto & ttl_entry : move_ttl_entries) updateTTL(ttl_entry, move_ttl_infos, move_ttl_infos.moves_ttl[ttl_entry.result_column], block, false); NamesAndTypesList columns = data.getColumns().getAllPhysical().filter(block.getNames()); @@ -287,9 +290,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa } if (data.hasRowsTTL()) - updateTTL(data.rows_ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true); + updateTTL(data.getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true); - for (const auto & [name, ttl_entry] : data.column_ttl_entries_by_name) + for (const auto & [name, ttl_entry] : data.getColumnTTLs()) updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true); new_data_part->ttl_infos.update(move_ttl_infos); diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 1609e1e517e..a2c2099a5f5 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -128,14 +128,14 @@ bool MergeTreePartsMover::selectPartsForMove( if (!can_move(part, &reason)) continue; - auto ttl_entry = part->storage.selectTTLEntryForTTLInfos(part->ttl_infos, time_of_move); + auto ttl_entry = data->selectTTLEntryForTTLInfos(part->ttl_infos, time_of_move); auto to_insert = need_to_move.find(part->volume->getDisk()); ReservationPtr reservation; if (ttl_entry) { - auto destination = ttl_entry->getDestination(policy); - if (destination && !ttl_entry->isPartInDestination(policy, *part)) - reservation = part->storage.tryReserveSpace(part->getBytesOnDisk(), ttl_entry->getDestination(policy)); + auto destination = data->getDestinationForTTL(*ttl_entry); + if (destination && !data->isPartInTTLDestination(*ttl_entry, *part)) + reservation = data->tryReserveSpace(part->getBytesOnDisk(), data->getDestinationForTTL(*ttl_entry)); } if (reservation) /// Found reservation by TTL rule. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index 5925c4a6935..12bec501181 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -53,7 +53,7 @@ ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTr if (data.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) partition_key = formattedAST(data.getPartitionKey().expression_list_ast); - ttl_table = formattedAST(data.ttl_table_ast); + ttl_table = formattedAST(data.getTableTTLs().definition_ast); skip_indices = data.getIndices().toString(); if (data.canUseAdaptiveGranularity()) diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index a25a3a25337..77fdfd60b46 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -39,8 +39,7 @@ public: return part->storage.mayBenefitFromIndexForIn(left_in_operand, query_context); } - bool hasAnyTTL() const override { return part->storage.hasAnyTTL(); } - bool hasRowsTTL() const override { return part->storage.hasRowsTTL(); } + //bool hasAnyTTL() const override { return part->storage.hasAnyTTL(); } ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const override { @@ -65,6 +64,8 @@ protected: setColumns(part_->storage.getColumns()); setIndices(part_->storage.getIndices()); setSortingKey(part_->storage.getSortingKey()); + setColumnTTLs(part->storage.getColumnTTLs()); + setTableTTLs(part->storage.getTableTTLs()); } private: diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 6c13033da1e..61982dcf46f 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -6,10 +6,21 @@ #include #include #include +#include +#include + +#include +#include namespace DB { + +namespace ErrorCodes +{ + extern const int BAD_TTL_EXPRESSION; +}; + StorageInMemoryMetadata::StorageInMemoryMetadata( const ColumnsDescription & columns_, const IndicesDescription & indices_, @@ -138,4 +149,63 @@ StorageMetadataKeyField StorageMetadataKeyField::getKeyFromAST(const ASTPtr & de return result; } + +namespace +{ + +void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name) +{ + for (const auto & action : ttl_expression->getActions()) + { + if (action.type == ExpressionAction::APPLY_FUNCTION) + { + IFunctionBase & func = *action.function_base; + if (!func.isDeterministic()) + throw Exception( + "TTL expression cannot contain non-deterministic functions, " + "but contains function " + + func.getName(), + ErrorCodes::BAD_ARGUMENTS); + } + } + + const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name); + + if (!typeid_cast(result_column.type.get()) + && !typeid_cast(result_column.type.get())) + { + throw Exception( + "TTL expression result column should have DateTime or Date type, but has " + result_column.type->getName(), + ErrorCodes::BAD_TTL_EXPRESSION); + } +} + +} + +StorageMetadataTTLField StorageMetadataTTLField::getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context) +{ + StorageMetadataTTLField result; + const auto * ttl_element = definition_ast->as(); + + /// First child is expression, like `TTL expr TO DISK` + if (ttl_element != nullptr) + result.definition_ast = ttl_element->children.front()->clone(); + else + result.definition_ast = definition_ast->clone(); + + auto ttl_ast = result.definition_ast->clone(); + auto syntax_result = SyntaxAnalyzer(context).analyze(ttl_ast, columns.getAllPhysical()); + result.expression = ExpressionAnalyzer(ttl_ast, syntax_result, context).getActions(false); + + if (ttl_element != nullptr) + { + result.destination_type = ttl_element->destination_type; + result.destination_name = ttl_element->destination_name; + } + + result.result_column = ttl_ast->getColumnName(); + + checkTTLExpression(result.expression, result.result_column); + return result; +} } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 3faaed8d680..76720248fbd 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -84,6 +84,20 @@ struct StorageMetadataTTLField DataDestinationType destination_type; String destination_name; + + static StorageMetadataTTLField getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context); +}; + +using StorageMetadataTTLColumnEntries = std::unordered_map; +using StorageMetadataTTLFields = std::vector; + +struct StorageMetadataTableTTL +{ + ASTPtr definition_ast; + + StorageMetadataTTLField rows_ttl; + + StorageMetadataTTLFields move_ttl; }; } From 62e9f676391dd0088c4ad69f8b4a4f1c78cb48a0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 May 2020 20:25:18 +0300 Subject: [PATCH 019/150] Better --- src/DataStreams/TTLBlockInputStream.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 99b185d39b9..00020be36bf 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -156,8 +156,7 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) } std::vector columns_to_remove; - const auto & column_ttl_entries_by_name = storage.getColumnTTLs(); - for (const auto & [name, ttl_entry] : column_ttl_entries_by_name) + for (const auto & [name, ttl_entry] : storage.getColumnTTLs()) { /// If we read not all table columns. E.g. while mutation. if (!block.has(name)) @@ -217,9 +216,8 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) void TTLBlockInputStream::updateMovesTTL(Block & block) { - const auto & move_ttl_entries = storage.getMoveTTLs(); std::vector columns_to_remove; - for (const auto & ttl_entry : move_ttl_entries) + for (const auto & ttl_entry : storage.getMoveTTLs()) { auto & new_ttl_info = new_ttl_infos.moves_ttl[ttl_entry.result_column]; From adc7439a4ca0e96fd1f82de9be09554679eb50d2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 May 2020 20:57:08 +0300 Subject: [PATCH 020/150] Comments and refactoring --- src/Storages/IStorage.cpp | 54 ++++++++++--------- src/Storages/IStorage.h | 33 ++++++------ src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 - src/Storages/StorageInMemoryMetadata.cpp | 11 ++-- src/Storages/StorageInMemoryMetadata.h | 22 +++++++- 6 files changed, 74 insertions(+), 50 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index f8f2599184f..6a0b6b46d5a 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -541,6 +541,35 @@ Names IStorage::getColumnsRequiredForSampling() const return {}; } +const StorageMetadataTableTTL & IStorage::getTableTTLs() const +{ + return table_ttl; +} + +void IStorage::setTableTTLs(const StorageMetadataTableTTL & table_ttl_) +{ + table_ttl = table_ttl_; +} + +bool IStorage::hasAnyTableTTL() const +{ + return hasAnyMoveTTL() || hasRowsTTL(); +} + +const StorageMetadataTTLColumnFields & IStorage::getColumnTTLs() const +{ + return column_ttls_by_name; +} + +void IStorage::setColumnTTLs(const StorageMetadataTTLColumnFields & column_ttls_by_name_) +{ + column_ttls_by_name = column_ttls_by_name_; +} + +bool IStorage::hasAnyColumnTTL() const +{ + return !column_ttls_by_name.empty(); +} const StorageMetadataTTLField & IStorage::getRowsTTL() const { @@ -562,29 +591,4 @@ bool IStorage::hasAnyMoveTTL() const return !table_ttl.move_ttl.empty(); } -const StorageMetadataTTLColumnEntries & IStorage::getColumnTTLs() const -{ - return column_ttls_by_name; -} - -void IStorage::setColumnTTLs(const StorageMetadataTTLColumnEntries & column_ttls_by_name_) -{ - column_ttls_by_name = column_ttls_by_name_; -} - -const StorageMetadataTableTTL & IStorage::getTableTTLs() const -{ - return table_ttl; -} - -void IStorage::setTableTTLs(const StorageMetadataTableTTL & table_ttl_) -{ - table_ttl = table_ttl_; -} - -bool IStorage::hasAnyTableTTL() const -{ - return hasAnyMoveTTL() || hasRowsTTL(); -} - } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 416d8ffbbb1..551117bd00a 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -130,7 +130,7 @@ public: virtual bool hasEvenlyDistributedRead() const { return false; } /// Returns true if there is set table TTL, any column TTL or any move TTL. - virtual bool hasAnyTTL() const { return hasRowsTTL() || hasAnyTableTTL(); } + virtual bool hasAnyTTL() const { return hasAnyColumnTTL() || hasAnyTableTTL(); } /// Optional size information of each physical column. /// Currently it's only used by the MergeTree family for query optimizations. @@ -202,7 +202,7 @@ private: StorageMetadataKeyField sorting_key; StorageMetadataKeyField sampling_key; - StorageMetadataTTLColumnEntries column_ttls_by_name; + StorageMetadataTTLColumnFields column_ttls_by_name; StorageMetadataTableTTL table_ttl; private: @@ -507,28 +507,31 @@ public: Names getColumnsRequiredForFinal() const { return getColumnsRequiredForSortingKey(); } - /// Returns columns, which will be needed to calculate dependencies - /// (skip indices, TTL expressions) if we update @updated_columns set of columns. + /// Returns columns, which will be needed to calculate dependencies (skip + /// indices, TTL expressions) if we update @updated_columns set of columns. virtual ColumnDependencies getColumnDependencies(const NameSet & /* updated_columns */) const { return {}; } - /// Returns storage policy if storage supports it + /// Returns storage policy if storage supports it. virtual StoragePolicyPtr getStoragePolicy() const { return {}; } - /// Returns true if there is set TTL for rows. - const StorageMetadataTTLField & getRowsTTL() const; - bool hasRowsTTL() const; - - const StorageMetadataTTLFields & getMoveTTLs() const; - bool hasAnyMoveTTL() const; - + /// Common tables TTLs (for rows and moves). const StorageMetadataTableTTL & getTableTTLs() const; void setTableTTLs(const StorageMetadataTableTTL & table_ttl_); bool hasAnyTableTTL() const; - const StorageMetadataTTLColumnEntries & getColumnTTLs() const; - void setColumnTTLs(const StorageMetadataTTLColumnEntries & column_ttls_by_name_); - bool hasAnyColumnTTL() const { return !column_ttls_by_name.empty(); } + /// Separate TTLs for columns. + const StorageMetadataTTLColumnFields & getColumnTTLs() const; + void setColumnTTLs(const StorageMetadataTTLColumnFields & column_ttls_by_name_); + bool hasAnyColumnTTL() const; + /// Just wrapper for table TTLs, return rows part of table TTLs. + const StorageMetadataTTLField & getRowsTTL() const; + bool hasRowsTTL() const; + + /// Just wrapper for table TTLs, return moves (to disks or volumes) parts of + /// table TTL. + const StorageMetadataTTLFields & getMoveTTLs() const; + bool hasAnyMoveTTL() const; /// If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it. /// Used for: diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fddd08716fd..737b90a56cf 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -588,7 +588,7 @@ void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, auto new_column_ttl_asts = new_columns.getColumnTTLs(); - StorageMetadataTTLColumnEntries new_column_ttl_by_name = getColumnTTLs(); + StorageMetadataTTLColumnFields new_column_ttl_by_name = getColumnTTLs(); if (!new_column_ttl_asts.empty()) { diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 77fdfd60b46..37acf0f0160 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -39,8 +39,6 @@ public: return part->storage.mayBenefitFromIndexForIn(left_in_operand, query_context); } - //bool hasAnyTTL() const override { return part->storage.hasAnyTTL(); } - ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const override { return part->storage.getColumnDependencies(updated_columns); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 61982dcf46f..7d9defef429 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -187,16 +187,17 @@ StorageMetadataTTLField StorageMetadataTTLField::getTTLFromAST(const ASTPtr & de StorageMetadataTTLField result; const auto * ttl_element = definition_ast->as(); - /// First child is expression, like `TTL expr TO DISK` + /// First child is expression: `TTL expr TO DISK` if (ttl_element != nullptr) - result.definition_ast = ttl_element->children.front()->clone(); - else - result.definition_ast = definition_ast->clone(); + result.expression_ast = ttl_element->children.front()->clone(); + else /// It's columns TTL without any additions, just copy it + result.expression_ast = definition_ast->clone(); - auto ttl_ast = result.definition_ast->clone(); + auto ttl_ast = result.expression_ast->clone(); auto syntax_result = SyntaxAnalyzer(context).analyze(ttl_ast, columns.getAllPhysical()); result.expression = ExpressionAnalyzer(ttl_ast, syntax_result, context).getActions(false); + /// Move TTL to disk or volume if (ttl_element != nullptr) { result.destination_type = ttl_element->destination_type; diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 76720248fbd..40864d79aad 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -73,30 +73,48 @@ struct StorageMetadataKeyField static StorageMetadataKeyField getKeyFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context); }; +/// Common struct for TTL record in storage struct StorageMetadataTTLField { - ASTPtr definition_ast; + /// Expression part of TTL AST: + /// TTL d + INTERVAL 1 DAY + /// ^~~~~expression~~~~^ + ASTPtr expression_ast; + /// Expresion actions evaluated from AST ExpressionActionsPtr expression; + /// Result column of this TTL expression String result_column; + /// Destination type, only valid for table TTLs. + /// For example DISK or VOLUME DataDestinationType destination_type; + /// Name of destination disk or volume String destination_name; + /// Parse TTL structure from definition. Able to parse both column and table + /// TTLs. static StorageMetadataTTLField getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context); }; -using StorageMetadataTTLColumnEntries = std::unordered_map; +/// Mapping from column name to column TTL +using StorageMetadataTTLColumnFields = std::unordered_map; using StorageMetadataTTLFields = std::vector; +/// Common TTL for all table. Specified after defining the table columns. struct StorageMetadataTableTTL { + /// Definition. Include all parts of TTL: + /// TTL d + INTERVAL 1 day TO VOLUME 'disk1' + /// ^~~~~~~~~~~~~~~definition~~~~~~~~~~~~~~~^ ASTPtr definition_ast; + /// Rows removing TTL StorageMetadataTTLField rows_ttl; + /// Moving data TTL (to other disks or volumes) StorageMetadataTTLFields move_ttl; }; From 211d7e229ff031ba4fa594afeb9d75e187c6054d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 May 2020 21:03:04 +0300 Subject: [PATCH 021/150] Better using --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 737b90a56cf..a58f176cffa 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2930,7 +2930,7 @@ std::optional MergeTreeData::selectTTLEntryForTTLInfos(const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t time_of_move) const { time_t max_max_ttl = 0; - std::vector::const_iterator best_entry_it; + StorageMetadataTTLFields::const_iterator best_entry_it; auto lock = std::lock_guard(move_ttl_entries_mutex); const auto & move_ttl_entries = getMoveTTLs(); From 355fbadb316e2a652a6ac8d3b405831416a736ec Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 25 May 2020 22:18:01 +0300 Subject: [PATCH 022/150] Fix style --- src/Storages/MergeTree/MergeTreeData.cpp | 1 - src/Storages/StorageInMemoryMetadata.cpp | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a58f176cffa..beeb163d5b5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -581,7 +581,6 @@ void MergeTreeData::initPartitionKey(ASTPtr partition_by_ast) } - void MergeTreeData::setTTLExpressions(const ColumnsDescription & new_columns, const ASTPtr & new_ttl_table_ast, bool only_check) { diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 7d9defef429..cf319ce319b 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -19,6 +19,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_TTL_EXPRESSION; + extern const int BAD_ARGUMENTS; }; StorageInMemoryMetadata::StorageInMemoryMetadata( From 3df27733419c7099d7dab7678667f4ab61f9231e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 26 May 2020 01:15:45 +0300 Subject: [PATCH 023/150] proofreading of 20.4 --- CHANGELOG.md | 68 +++++++++++++++++++++++++--------------------------- 1 file changed, 33 insertions(+), 35 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 61ca07f9bb9..6cbf3d96baf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,46 +4,44 @@ #### Bug Fix -* Remove logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). +* Removed logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). * Fixed memory leak in registerDiskS3. [#11074](https://github.com/ClickHouse/ClickHouse/pull/11074) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fixes the potential missed data during termination of Kafka engine table. [#11048](https://github.com/ClickHouse/ClickHouse/pull/11048) ([filimonov](https://github.com/filimonov)). -* Fixed parseDateTime64BestEffort argument resolution bugs. ... [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). -* Fix very rare potential use-after-free error in MergeTree if table was not created successfully. [#10986](https://github.com/ClickHouse/ClickHouse/pull/10986) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix metadata (relative path for rename) and data (relative path for symlink) handling for Atomic database. [#10980](https://github.com/ClickHouse/ClickHouse/pull/10980) ([Azat Khuzhin](https://github.com/azat)). -* Fix very rare potential use-after-free error in MergeTree if table was not created successfully. [#10970](https://github.com/ClickHouse/ClickHouse/pull/10970) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix server crash on concurrent `ALTER` and `DROP DATABASE` queries with `Atomic` database engine. [#10968](https://github.com/ClickHouse/ClickHouse/pull/10968) ([tavplubix](https://github.com/tavplubix)). -* Fix incorrect raw data size in method getRawData(). [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). -* Fix incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Avoid sending partially written files by the DistributedBlockOutputStream. [#10940](https://github.com/ClickHouse/ClickHouse/pull/10940) ([Azat Khuzhin](https://github.com/azat)). -* Fix crash in `SELECT count(notNullIn(NULL, []))`. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix for the hang which was happening sometimes during DROP of table engine=Kafka (or during server restarts). [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). -* Now it's possible to execute multiple `ALTER RENAME` like `a TO b, c TO a`. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). -* Fix possible race which could happen when you get result from aggregate function state from multiple thread for the same column. The only way (which I found) it can happen is when you use `finalizeAggregation` function while reading from table with `Memory` engine which stores `AggregateFunction` state for `quanite*` function. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). -* Fix SIGSEGV in StringHashTable (if such key does not exist). [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the potential missed data during termination of Kafka engine table. [#11048](https://github.com/ClickHouse/ClickHouse/pull/11048) ([filimonov](https://github.com/filimonov)). +* Fixed `parseDateTime64BestEffort` argument resolution bugs. [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). +* Fixed very rare potential use-after-free error in `MergeTree` if table was not created successfully. [#10986](https://github.com/ClickHouse/ClickHouse/pull/10986), [#10970](https://github.com/ClickHouse/ClickHouse/pull/10970) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed metadata (relative path for rename) and data (relative path for symlink) handling for Atomic database. [#10980](https://github.com/ClickHouse/ClickHouse/pull/10980) ([Azat Khuzhin](https://github.com/azat)). +* Fixed server crash on concurrent `ALTER` and `DROP DATABASE` queries with `Atomic` database engine. [#10968](https://github.com/ClickHouse/ClickHouse/pull/10968) ([tavplubix](https://github.com/tavplubix)). +* Fixed incorrect raw data size in `getRawData()` method. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). +* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed sending partially written files by the `DistributedBlockOutputStream`. [#10940](https://github.com/ClickHouse/ClickHouse/pull/10940) ([Azat Khuzhin](https://github.com/azat)). +* Fixed crash in `SELECT count(notNullIn(NULL, []))`. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the hang which was happening sometimes during `DROP` of `Kafka` table engine. (or during server restarts). [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). +* Fixed the impossibility of executing multiple `ALTER RENAME` like `a TO b, c TO a`. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). +* Fixed possible race which could happen when you get result from aggregate function state from multiple thread for the same column. The only way it can happen is when you use `finalizeAggregation` function while reading from table with `Memory` engine which stores `AggregateFunction` state for `quanite*` function. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed `SIGSEGV` in `StringHashTable` if such a key does not exist. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). * Fixed `WATCH` hangs after `LiveView` table was dropped from database with `Atomic` engine. [#10859](https://github.com/ClickHouse/ClickHouse/pull/10859) ([tavplubix](https://github.com/tavplubix)). * Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). * Now constraints are updated if the column participating in `CONSTRAINT` expression was renamed. Fixes [#10844](https://github.com/ClickHouse/ClickHouse/issues/10844). [#10847](https://github.com/ClickHouse/ClickHouse/pull/10847) ([alesapin](https://github.com/alesapin)). -* Fix potential read of uninitialized memory in cache dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix columns order after Block::sortColumns() (also add a test that shows that it affects some real use case - Buffer engine). [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). -* Fix the issue with ODBC bridge when no quoting of identifiers is requested. This fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix UBSan and MSan report in DateLUT. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* -. [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). -* Fix `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix combinator -OrNull and -OrDefault when combined with -State. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). -* Fix possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). -* Fix nullptr dereference in StorageBuffer if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix optimize_skip_unused_shards with LowCardinality. [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). +* Fixed potential read of uninitialized memory in cache-dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed columns order after `Block::sortColumns()`. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the issue with `ODBC` bridge when no quoting of identifiers is requested. Fixes [#7984] (https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `UBSan` and `MSan` report in `DateLUT`. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). +* Fixed `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed combinator `-OrNull` and `-OrDefault` when combined with `-State`. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). +* Fixed possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). +* Fixed `nullpt`r dereference in `StorageBuffer` if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `optimize_skip_unused_shards` with `LowCardinality`. [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). * Fixed handling condition variable for synchronous mutations. In some cases signals to that condition variable could be lost. [#10588](https://github.com/ClickHouse/ClickHouse/pull/10588) ([Vladimir Chebotarev](https://github.com/excitoon)). -* This PR fixes possible crash when `createDictionary()` is called before `loadStoredObject()` has finished. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix SELECT of column ALIAS which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* * implemented comparison between DateTime64 and String values (just like for DateTime). ... [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). -* Disable GROUP BY sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for WITH ROLLUP/CUBE/TOTALS. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). -* Fixes: [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263) (after that PR dist send via INSERT had been postponing on each INSERT) Fixes: [#8756](https://github.com/ClickHouse/ClickHouse/issues/8756) (that PR breaks distributed sends with all of the following conditions met (unlikely setup for now I guess): `internal_replication == false`, multiple local shards (activates the hardlinking code) and `distributed_storage_policy` (makes `link(2)` fails on `EXDEV`)). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). -* Todo. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -* #10551. [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed possible crash when `createDictionary()` is called before `loadStoredObject()` has finished. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed `SELECT` of column `ALIAS` which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). +* Fixed implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). +* Disable `GROUP BY` sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for `WITH ROLLUP/CUBE/TOTALS`. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). +* Fixed [#10263] (https://github.com/ClickHouse/ClickHouse/issues/10263). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). +* Added failing tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added backward compatibility for create bloom filter index. [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). ### ClickHouse release v20.4.2.9, 2020-05-12 From 7bbb064b34a9bab7a21d736ecbc00eea633cbf93 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 May 2020 11:58:07 +0300 Subject: [PATCH 024/150] Fix metadata update --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.h | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index beeb163d5b5..cb7d8257f05 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -259,8 +259,8 @@ StorageInMemoryMetadata MergeTreeData::getInMemoryMetadata() const if (isPrimaryKeyDefined()) metadata.primary_key_ast = getPrimaryKeyAST()->clone(); - if (ttl_table_ast) - metadata.ttl_for_table_ast = ttl_table_ast->clone(); + if (hasAnyTableTTL()) + metadata.ttl_for_table_ast = getTableTTLs().definition_ast->clone(); if (hasSamplingKey()) metadata.sample_by_ast = getSamplingKeyAST()->clone(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index a94680658ff..c4ab18b6099 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -682,7 +682,6 @@ protected: friend struct ReplicatedMergeTreeTableMetadata; friend class StorageReplicatedMergeTree; - ASTPtr ttl_table_ast; ASTPtr settings_ast; bool require_part_metadata; From 6626ef08e83c27ee6cb12eb6a15e90083cdbdbff Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 May 2020 15:14:50 +0300 Subject: [PATCH 025/150] Fix PVS studio errors --- src/Storages/IStorage.h | 4 +++- src/Storages/MergeTree/MergeTreeData.cpp | 3 --- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 551117bd00a..fc99c1a5f78 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -82,7 +82,9 @@ class IStorage : public std::enable_shared_from_this, public TypePromo { public: IStorage() = delete; - explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)) {} + /// Storage fields should be initialized in separate methods like setColumns + /// or setTableTTLs. + explicit IStorage(StorageID storage_id_) : storage_id(std::move(storage_id_)) {} //-V730 virtual ~IStorage() = default; IStorage(const IStorage &) = delete; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index cb7d8257f05..92a23e172a9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2481,9 +2481,6 @@ void MergeTreeData::movePartitionToDisk(const ASTPtr & partition, const String & return part_ptr->volume->getDisk()->getName() == disk->getName(); }), parts.end()); - if (parts.empty()) - throw Exception("Nothing to move", ErrorCodes::NO_SUCH_DATA_PART); - if (parts.empty()) { String no_parts_to_move_message; From 8614487a5a44b833cf808bcd95cf312f2ab4eecf Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 26 May 2020 16:46:19 +0300 Subject: [PATCH 026/150] Fix primary key check --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 12baef4b7be..dcbda0a2c80 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -837,9 +837,10 @@ void IMergeTreeDataPart::checkConsistencyBase() const { String path = getFullRelativePath(); + const auto & pk = storage.getPrimaryKey(); if (!checksums.empty()) { - if (storage.hasPrimaryKey() && !checksums.files.count("primary.idx")) + if (!pk.column_names.empty() && !checksums.files.count("primary.idx")) throw Exception("No checksum for primary.idx", ErrorCodes::NO_FILE_IN_DATA_PART); if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) @@ -873,7 +874,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const }; /// Check that the primary key index is not empty. - if (storage.hasPrimaryKey()) + if (!pk.column_names.empty()) check_file_not_empty(volume->getDisk(), path + "primary.idx"); if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) From 2b07f69e5714022983277778e4c9d9e275685517 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 27 May 2020 18:02:25 +0300 Subject: [PATCH 027/150] S3 Poco HTTP Client WIP. --- src/CMakeLists.txt | 1 + src/IO/S3/PocoHttpClient.cpp | 139 ++++++++++++++++++++++++++++ src/IO/S3/PocoHttpClient.h | 36 +++++++ src/IO/S3/PocoHttpClientFactory.cpp | 32 +++++++ src/IO/S3/PocoHttpClientFactory.h | 25 +++++ src/IO/S3Common.cpp | 6 ++ 6 files changed, 239 insertions(+) create mode 100644 src/IO/S3/PocoHttpClient.cpp create mode 100644 src/IO/S3/PocoHttpClient.h create mode 100644 src/IO/S3/PocoHttpClientFactory.cpp create mode 100644 src/IO/S3/PocoHttpClientFactory.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 222a3e486f9..3e48271bb51 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -73,6 +73,7 @@ if(USE_RDKAFKA) endif() if (USE_AWS_S3) + add_headers_and_sources(dbms Common/S3) add_headers_and_sources(dbms Disks/S3) endif() diff --git a/src/IO/S3/PocoHttpClient.cpp b/src/IO/S3/PocoHttpClient.cpp new file mode 100644 index 00000000000..397d278527f --- /dev/null +++ b/src/IO/S3/PocoHttpClient.cpp @@ -0,0 +1,139 @@ +#include "PocoHttpClient.h" + +#include +#include +#include +#include +#include +#include +#include +#include "Poco/StreamCopier.h" +#include +#include +#include + +namespace DB::S3 +{ +std::shared_ptr PocoHttpClient::MakeRequest( + Aws::Http::HttpRequest & request, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const +{ + auto response = Aws::MakeShared("PocoHttpClient", request); + MakeRequestInternal(request, response, readLimiter, writeLimiter); + return response; +} + +std::shared_ptr PocoHttpClient::MakeRequest( + const std::shared_ptr & request, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const +{ + auto response = Aws::MakeShared("PocoHttpClient", request); + MakeRequestInternal(*request, response, readLimiter, writeLimiter); + return response; +} + +void PocoHttpClient::MakeRequestInternal( + Aws::Http::HttpRequest & request, + std::shared_ptr & response, + Aws::Utils::RateLimits::RateLimiterInterface *, + Aws::Utils::RateLimits::RateLimiterInterface *) const +{ + auto uri = request.GetUri().GetURIString(); + + LOG_DEBUG(&Logger::get("AWSClient"), "Make request to: {}", uri); + + /// 1 second is enough for now. + /// TODO: Make timeouts configurable. + ConnectionTimeouts timeouts( + Poco::Timespan(1000000), /// Connection timeout. + Poco::Timespan(1000000), /// Send timeout. + Poco::Timespan(1000000) /// Receive timeout. + ); + auto session = makeHTTPSession(Poco::URI(uri), timeouts); + + LOG_DEBUG(&Logger::get("AWSClient"), "Here 1"); + + try + { + Poco::Net::HTTPRequest request_(Poco::Net::HTTPRequest::HTTP_1_1); + + request_.setURI(uri); + + LOG_DEBUG(&Logger::get("AWSClient"), "Here 2"); + + switch (request.GetMethod()) + { + case Aws::Http::HttpMethod::HTTP_GET: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_GET); + break; + case Aws::Http::HttpMethod::HTTP_POST: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_POST); + break; + case Aws::Http::HttpMethod::HTTP_DELETE: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_DELETE); + break; + case Aws::Http::HttpMethod::HTTP_PUT: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_PUT); + break; + case Aws::Http::HttpMethod::HTTP_HEAD: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_HEAD); + break; + case Aws::Http::HttpMethod::HTTP_PATCH: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_PATCH); + break; + } + + LOG_DEBUG(&Logger::get("AWSClient"), "Here 3"); + + for (const auto & [header_name, header_value] : request.GetHeaders()) + request_.set(header_name, header_value); + + LOG_DEBUG(&Logger::get("AWSClient"), "Here 4"); + + auto & request_body_stream = session->sendRequest(request_); + + if (request.GetContentBody()) + Poco::StreamCopier::copyStream(*(request.GetContentBody()), request_body_stream); + else + LOG_ERROR(&Logger::get("AWSClient"), "No content body :("); + + LOG_DEBUG(&Logger::get("AWSClient"), "Here 5"); + + Poco::Net::HTTPResponse response_; + auto & response_body_stream = session->receiveResponse(response_); + + LOG_DEBUG(&Logger::get("AWSClient"), "Here 6"); + + response->SetResponseCode(static_cast(response_.getStatus())); + + LOG_DEBUG(&Logger::get("AWSClient"), "Here 7"); + + response->SetContentType(response_.getContentType()); + + LOG_DEBUG(&Logger::get("AWSClient"), "Content length: {}", response_.getContentLength()); + + LOG_DEBUG(&Logger::get("AWSClient"), "Received headers:"); + for (auto it = response_.begin(); it != response_.end(); ++it) + { + LOG_DEBUG(&Logger::get("AWSClient"), "{} : {}", it->first, it->second); + } + + /// TODO: Do not copy whole stream. + Poco::StreamCopier::copyStream(response_body_stream, response->GetResponseBody()); + + if (response_.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) + { + response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); + response->SetClientErrorMessage(response_.getReason()); + } + } + catch (...) + { + tryLogCurrentException(&Logger::get("AWSClient"), "Failed to make request"); + response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); + response->SetClientErrorMessage(getCurrentExceptionMessage(true)); + } +} +} diff --git a/src/IO/S3/PocoHttpClient.h b/src/IO/S3/PocoHttpClient.h new file mode 100644 index 00000000000..5de20c9d978 --- /dev/null +++ b/src/IO/S3/PocoHttpClient.h @@ -0,0 +1,36 @@ +#pragma once + +#include + +namespace Aws::Http::Standard +{ +class StandardHttpResponse; +} + +namespace DB::S3 +{ + +class PocoHttpClient : public Aws::Http::HttpClient +{ +public: + ~PocoHttpClient() override = default; + std::shared_ptr MakeRequest( + Aws::Http::HttpRequest & request, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const override; + + std::shared_ptr MakeRequest( + const std::shared_ptr & request, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const override; + +private: + void MakeRequestInternal( + Aws::Http::HttpRequest & request, + std::shared_ptr & response, + Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, + Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; + +}; + +} diff --git a/src/IO/S3/PocoHttpClientFactory.cpp b/src/IO/S3/PocoHttpClientFactory.cpp new file mode 100644 index 00000000000..e72f9a8a4a4 --- /dev/null +++ b/src/IO/S3/PocoHttpClientFactory.cpp @@ -0,0 +1,32 @@ +#include "PocoHttpClientFactory.h" + +#include +#include +#include +#include +#include + +namespace DB::S3 +{ +std::shared_ptr +PocoHttpClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration &) const +{ + return std::make_shared(); +} + +std::shared_ptr PocoHttpClientFactory::CreateHttpRequest( + const Aws::String & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const +{ + return CreateHttpRequest(Aws::Http::URI(uri), method, streamFactory); +} + +std::shared_ptr PocoHttpClientFactory::CreateHttpRequest( + const Aws::Http::URI & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const +{ + auto request = Aws::MakeShared("PocoHttpClientFactory", uri, method); + request->SetResponseStreamFactory(streamFactory); + + return request; +} + +} diff --git a/src/IO/S3/PocoHttpClientFactory.h b/src/IO/S3/PocoHttpClientFactory.h new file mode 100644 index 00000000000..ac73a0356ff --- /dev/null +++ b/src/IO/S3/PocoHttpClientFactory.h @@ -0,0 +1,25 @@ +#pragma once + +#include + +namespace Aws::Http +{ + class HttpClient; + class HttpRequest; +} + +namespace DB::S3 +{ + +class PocoHttpClientFactory : public Aws::Http::HttpClientFactory +{ +public: + ~PocoHttpClientFactory() override = default; + [[nodiscard]] std::shared_ptr CreateHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration) const override; + [[nodiscard]] std::shared_ptr + CreateHttpRequest(const Aws::String & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const override; + [[nodiscard]] std::shared_ptr + CreateHttpRequest(const Aws::Http::URI & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const override; +}; + +} diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 3c2c5835078..1a007bf2a46 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -9,6 +9,11 @@ # include # include # include +# include +# include +# include +# include +# include # include # include # include @@ -70,6 +75,7 @@ namespace S3 aws_options = Aws::SDKOptions{}; Aws::InitAPI(aws_options); Aws::Utils::Logging::InitializeAWSLogging(std::make_shared()); + Aws::Http::SetHttpClientFactory(std::make_shared()); } ClientFactory::~ClientFactory() From a2dccc778ca018289d22daec75e7f291de60a624 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 27 May 2020 19:56:17 +0300 Subject: [PATCH 028/150] S3 Poco HTTP Client WIP (redirect support). --- src/IO/S3/PocoHttpClient.cpp | 143 +++++++++++++++++++---------------- 1 file changed, 77 insertions(+), 66 deletions(-) diff --git a/src/IO/S3/PocoHttpClient.cpp b/src/IO/S3/PocoHttpClient.cpp index 397d278527f..fa431adc73c 100644 --- a/src/IO/S3/PocoHttpClient.cpp +++ b/src/IO/S3/PocoHttpClient.cpp @@ -44,94 +44,105 @@ void PocoHttpClient::MakeRequestInternal( LOG_DEBUG(&Logger::get("AWSClient"), "Make request to: {}", uri); - /// 1 second is enough for now. - /// TODO: Make timeouts configurable. - ConnectionTimeouts timeouts( - Poco::Timespan(1000000), /// Connection timeout. - Poco::Timespan(1000000), /// Send timeout. - Poco::Timespan(1000000) /// Receive timeout. - ); - auto session = makeHTTPSession(Poco::URI(uri), timeouts); - - LOG_DEBUG(&Logger::get("AWSClient"), "Here 1"); - + const int MAX_REDIRECT_ATTEMPTS = 10; try { - Poco::Net::HTTPRequest request_(Poco::Net::HTTPRequest::HTTP_1_1); - - request_.setURI(uri); - - LOG_DEBUG(&Logger::get("AWSClient"), "Here 2"); - - switch (request.GetMethod()) + for (int attempt = 0; attempt < MAX_REDIRECT_ATTEMPTS; ++attempt) { - case Aws::Http::HttpMethod::HTTP_GET: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_GET); - break; - case Aws::Http::HttpMethod::HTTP_POST: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_POST); - break; - case Aws::Http::HttpMethod::HTTP_DELETE: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_DELETE); - break; - case Aws::Http::HttpMethod::HTTP_PUT: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_PUT); - break; - case Aws::Http::HttpMethod::HTTP_HEAD: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_HEAD); - break; - case Aws::Http::HttpMethod::HTTP_PATCH: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_PATCH); - break; - } + /// 1 second is enough for now. + /// TODO: Make timeouts configurable. + ConnectionTimeouts timeouts( + Poco::Timespan(2000000), /// Connection timeout. + Poco::Timespan(2000000), /// Send timeout. + Poco::Timespan(2000000) /// Receive timeout. + ); + auto session = makeHTTPSession(Poco::URI(uri), timeouts); - LOG_DEBUG(&Logger::get("AWSClient"), "Here 3"); + Poco::Net::HTTPRequest request_(Poco::Net::HTTPRequest::HTTP_1_1); - for (const auto & [header_name, header_value] : request.GetHeaders()) - request_.set(header_name, header_value); + request_.setURI(uri); - LOG_DEBUG(&Logger::get("AWSClient"), "Here 4"); + switch (request.GetMethod()) + { + case Aws::Http::HttpMethod::HTTP_GET: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_GET); + break; + case Aws::Http::HttpMethod::HTTP_POST: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_POST); + break; + case Aws::Http::HttpMethod::HTTP_DELETE: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_DELETE); + break; + case Aws::Http::HttpMethod::HTTP_PUT: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_PUT); + break; + case Aws::Http::HttpMethod::HTTP_HEAD: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_HEAD); + break; + case Aws::Http::HttpMethod::HTTP_PATCH: + request_.setMethod(Poco::Net::HTTPRequest::HTTP_PATCH); + break; + } - auto & request_body_stream = session->sendRequest(request_); + for (const auto & [header_name, header_value] : request.GetHeaders()) + request_.set(header_name, header_value); - if (request.GetContentBody()) - Poco::StreamCopier::copyStream(*(request.GetContentBody()), request_body_stream); - else - LOG_ERROR(&Logger::get("AWSClient"), "No content body :("); + auto & request_body_stream = session->sendRequest(request_); - LOG_DEBUG(&Logger::get("AWSClient"), "Here 5"); + if (request.GetContentBody()) + { + if (attempt > 0) /// Rewind buffer if it's not first attempt to write. + { + request.GetContentBody()->clear(); + request.GetContentBody()->seekg(0); + } + auto sz = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); + LOG_DEBUG( + &Logger::get("AWSClient"), "Written {} bytes to request body", sz); + } - Poco::Net::HTTPResponse response_; - auto & response_body_stream = session->receiveResponse(response_); + Poco::Net::HTTPResponse response_; + auto & response_body_stream = session->receiveResponse(response_); - LOG_DEBUG(&Logger::get("AWSClient"), "Here 6"); + LOG_DEBUG( + &Logger::get("AWSClient"), "Response status: {}, {}", static_cast(response_.getStatus()), response_.getReason()); - response->SetResponseCode(static_cast(response_.getStatus())); + if (response_.getStatus() == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) + { + auto location = response_.get("location"); + uri = location; + LOG_DEBUG(&Logger::get("AWSClient"), "Redirecting request to new location: {}", location); - LOG_DEBUG(&Logger::get("AWSClient"), "Here 7"); + continue; + } - response->SetContentType(response_.getContentType()); + response->SetResponseCode(static_cast(response_.getStatus())); + response->SetContentType(response_.getContentType()); - LOG_DEBUG(&Logger::get("AWSClient"), "Content length: {}", response_.getContentLength()); + std::stringstream headers_ss; + for (const auto & [header_name, header_value] : response_) + { + response->AddHeader(header_name, header_value); + headers_ss << " " << header_name << " : " << header_value << ";"; + } - LOG_DEBUG(&Logger::get("AWSClient"), "Received headers:"); - for (auto it = response_.begin(); it != response_.end(); ++it) - { - LOG_DEBUG(&Logger::get("AWSClient"), "{} : {}", it->first, it->second); - } + LOG_DEBUG(&Logger::get("AWSClient"), "Received headers:{}", headers_ss.str()); - /// TODO: Do not copy whole stream. - Poco::StreamCopier::copyStream(response_body_stream, response->GetResponseBody()); + /// TODO: Do not copy whole stream. + Poco::StreamCopier::copyStream(response_body_stream, response->GetResponseBody()); - if (response_.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) - { - response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); - response->SetClientErrorMessage(response_.getReason()); + if (response_.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) + { + response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); + response->SetClientErrorMessage(response_.getReason()); + } + + break; } } catch (...) { - tryLogCurrentException(&Logger::get("AWSClient"), "Failed to make request"); + tryLogCurrentException(&Logger::get("AWSClient"), "Failed to make request to: " + uri); response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); response->SetClientErrorMessage(getCurrentExceptionMessage(true)); } From 677cb56a06f7430f79383beb817760ee17f87860 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 27 May 2020 22:41:49 +0300 Subject: [PATCH 029/150] S3 Poco HTTP Client WIP. --- src/IO/S3/PocoHttpClient.cpp | 33 +++++++++++++++++---------------- src/Storages/StorageS3.cpp | 1 - 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/IO/S3/PocoHttpClient.cpp b/src/IO/S3/PocoHttpClient.cpp index fa431adc73c..cad100ab717 100644 --- a/src/IO/S3/PocoHttpClient.cpp +++ b/src/IO/S3/PocoHttpClient.cpp @@ -87,25 +87,26 @@ void PocoHttpClient::MakeRequestInternal( for (const auto & [header_name, header_value] : request.GetHeaders()) request_.set(header_name, header_value); - auto & request_body_stream = session->sendRequest(request_); - - if (request.GetContentBody()) - { - if (attempt > 0) /// Rewind buffer if it's not first attempt to write. - { - request.GetContentBody()->clear(); - request.GetContentBody()->seekg(0); - } - auto sz = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); - LOG_DEBUG( - &Logger::get("AWSClient"), "Written {} bytes to request body", sz); - } + request_.setExpectContinue(true); Poco::Net::HTTPResponse response_; + auto & request_body_stream = session->sendRequest(request_); + + if (session->peekResponse(response_)) + { + if (request.GetContentBody()) + { + auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); + LOG_DEBUG( + &Logger::get("AWSClient"), "Written {} bytes to request body", size); + } + } + auto & response_body_stream = session->receiveResponse(response_); + int status_code = static_cast(response_.getStatus()); LOG_DEBUG( - &Logger::get("AWSClient"), "Response status: {}, {}", static_cast(response_.getStatus()), response_.getReason()); + &Logger::get("AWSClient"), "Response status: {}, {}", status_code, response_.getReason()); if (response_.getStatus() == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) { @@ -116,7 +117,7 @@ void PocoHttpClient::MakeRequestInternal( continue; } - response->SetResponseCode(static_cast(response_.getStatus())); + response->SetResponseCode(static_cast(status_code)); response->SetContentType(response_.getContentType()); std::stringstream headers_ss; @@ -131,7 +132,7 @@ void PocoHttpClient::MakeRequestInternal( /// TODO: Do not copy whole stream. Poco::StreamCopier::copyStream(response_body_stream, response->GetResponseBody()); - if (response_.getStatus() != Poco::Net::HTTPResponse::HTTP_OK) + if (status_code >= 300) { response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); response->SetClientErrorMessage(response_.getReason()); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index b8710de507b..f958218f51f 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -119,7 +119,6 @@ namespace return Chunk(std::move(columns), num_rows); } - reader->readSuffix(); reader.reset(); return {}; From e3c7ab2ca7314a1c52a2728f9fad8a918fc96179 Mon Sep 17 00:00:00 2001 From: madianjun Date: Tue, 19 May 2020 17:31:55 +0800 Subject: [PATCH 030/150] Add Kafka client.id --- src/Storages/Kafka/KafkaSettings.h | 1 + src/Storages/Kafka/StorageKafka.cpp | 18 +++++++++++++++--- src/Storages/Kafka/StorageKafka.h | 2 ++ 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 93983fb6080..43984f81e05 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -19,6 +19,7 @@ struct KafkaSettings : public SettingsCollection M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.", 0) \ M(SettingString, kafka_topic_list, "", "A list of Kafka topics.", 0) \ M(SettingString, kafka_group_name, "", "A group of Kafka consumers.", 0) \ + M(SettingString, kafka_client_id, "", "A client id of Kafka consumer.", 0) \ M(SettingString, kafka_format, "", "The message format for Kafka engine.", 0) \ M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.", 0) \ M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 7731cf3c06a..a94aaca841b 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -118,6 +118,7 @@ StorageKafka::StorageKafka( const ColumnsDescription & columns_, const String & brokers_, const String & group_, + const String & client_id_, const Names & topics_, const String & format_name_, char row_delimiter_, @@ -132,6 +133,7 @@ StorageKafka::StorageKafka( , topics(global_context.getMacros()->expand(topics_)) , brokers(global_context.getMacros()->expand(brokers_)) , group(global_context.getMacros()->expand(group_)) + , client_id(global_context.getMacros()->expand(client_id_)) , format_name(global_context.getMacros()->expand(format_name_)) , row_delimiter(row_delimiter_) , schema_name(global_context.getMacros()->expand(schema_name_)) @@ -262,7 +264,7 @@ ProducerBufferPtr StorageKafka::createWriteBuffer(const Block & header) cppkafka::Configuration conf; conf.set("metadata.broker.list", brokers); conf.set("group.id", group); - conf.set("client.id", VERSION_FULL); + conf.set("client.id", client_id); // TODO: fill required settings updateConfiguration(conf); @@ -281,7 +283,7 @@ ConsumerBufferPtr StorageKafka::createReadBuffer() conf.set("metadata.broker.list", brokers); conf.set("group.id", group); - conf.set("client.id", VERSION_FULL); + conf.set("client.id", client_id); conf.set("auto.offset.reset", "smallest"); // If no offset stored for this group, read all messages from the start @@ -503,6 +505,7 @@ void registerStorageKafka(StorageFactory & factory) * - Kafka broker list * - List of topics * - Group ID (may be a constaint expression with a string result) + * - Client ID * - Message format (string) * - Row delimiter * - Schema (optional, if the format supports it) @@ -545,6 +548,7 @@ void registerStorageKafka(StorageFactory & factory) CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size) CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages) CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch) + CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id) #undef CHECK_KAFKA_STORAGE_ARGUMENT @@ -709,9 +713,17 @@ void registerStorageKafka(StorageFactory & factory) } } + // Get and check client id + String client_id = kafka_settings.kafka_client_id.value; + if (args_count >= 11) + { + engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); + client_id = engine_args[10]->as().value.safeGet(); + } + return StorageKafka::create( args.table_id, args.context, args.columns, - brokers, group, topics, format, row_delimiter, schema, num_consumers, max_block_size, skip_broken, intermediate_commit); + brokers, group, client_id, topics, format, row_delimiter, schema, num_consumers, max_block_size, skip_broken, intermediate_commit); }; factory.registerStorage("Kafka", creator_fn, StorageFactory::StorageFeatures{ .supports_settings = true, }); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index c813ed0033d..0831218a534 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -67,6 +67,7 @@ protected: const ColumnsDescription & columns_, const String & brokers_, const String & group_, + const String & client_id_, const Names & topics_, const String & format_name_, char row_delimiter_, @@ -83,6 +84,7 @@ private: Names topics; const String brokers; const String group; + const String client_id; const String format_name; char row_delimiter; /// optional row delimiter for generating char delimited stream in order to make various input stream parsers happy. const String schema_name; From 1fa795988faea4a8f76383f67252ccca373be5ab Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 11:24:59 +0300 Subject: [PATCH 031/150] Remove some code. --- src/Interpreters/IInterpreter.h | 4 - src/Interpreters/InterpreterSelectQuery.cpp | 878 ++---------------- src/Interpreters/InterpreterSelectQuery.h | 95 +- .../InterpreterSelectWithUnionQuery.cpp | 73 +- .../InterpreterSelectWithUnionQuery.h | 6 - src/Interpreters/executeQuery.cpp | 9 +- src/Storages/IStorage.cpp | 19 - src/Storages/IStorage.h | 10 - src/Storages/SelectQueryInfo.h | 22 - src/Storages/StorageView.cpp | 12 +- 10 files changed, 100 insertions(+), 1028 deletions(-) diff --git a/src/Interpreters/IInterpreter.h b/src/Interpreters/IInterpreter.h index 2425d015e56..63f36f1b18a 100644 --- a/src/Interpreters/IInterpreter.h +++ b/src/Interpreters/IInterpreter.h @@ -22,10 +22,6 @@ public: */ virtual BlockIO execute() = 0; - virtual QueryPipeline executeWithProcessors() { throw Exception("executeWithProcessors not implemented", ErrorCodes::NOT_IMPLEMENTED); } - - virtual bool canExecuteWithProcessors() const { return false; } - virtual bool ignoreQuota() const { return false; } virtual bool ignoreLimits() const { return false; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 4f717eda706..3af128ffb00 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1,39 +1,15 @@ #include -#include -#include -#include -#include -#include -#include -#include #include #include -#include -#include -#include -#include -#include -#include -#include #include #include -#include -#include -#include -#include -#include #include -#include -#include -#include - #include #include #include #include #include #include -#include #include #include @@ -59,7 +35,6 @@ #include #include -#include #include #include @@ -68,7 +43,6 @@ #include #include #include -#include #include #include #include @@ -98,7 +72,7 @@ #include #include #include -#include +#include namespace DB @@ -465,38 +439,13 @@ Block InterpreterSelectQuery::getSampleBlock() BlockIO InterpreterSelectQuery::execute() { - Pipeline pipeline; BlockIO res; - executeImpl(pipeline, input, std::move(input_pipe), res.pipeline); - executeUnion(pipeline, getSampleBlock()); - - res.in = pipeline.firstStream(); + executeImpl(res.pipeline, input, std::move(input_pipe)); res.pipeline.addInterpreterContext(context); res.pipeline.addStorageHolder(storage); return res; } -BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams(QueryPipeline & parent_pipeline) -{ - ///FIXME pipeline must be alive until query is finished - Pipeline pipeline; - executeImpl(pipeline, input, std::move(input_pipe), parent_pipeline); - unifyStreams(pipeline, getSampleBlock()); - parent_pipeline.addInterpreterContext(context); - parent_pipeline.addStorageHolder(storage); - return pipeline.streams; -} - -QueryPipeline InterpreterSelectQuery::executeWithProcessors() -{ - QueryPipeline query_pipeline; - executeImpl(query_pipeline, input, std::move(input_pipe), query_pipeline); - query_pipeline.addInterpreterContext(context); - query_pipeline.addStorageHolder(storage); - return query_pipeline; -} - - Block InterpreterSelectQuery::getSampleBlockImpl() { if (storage && !options.only_analyze) @@ -687,9 +636,7 @@ static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & c return 0; } - -template -void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe, QueryPipeline & save_context_and_storage) +void InterpreterSelectQuery::executeImpl(QueryPipeline & pipeline, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe) { /** Streams of data. When the query is executed in parallel, we have several data streams. * If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then @@ -701,8 +648,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS * then perform the remaining operations with one resulting stream. */ - constexpr bool pipeline_with_processors = std::is_same::value; - /// Now we will compose block streams that perform the necessary actions. auto & query = getSelectQuery(); const Settings & settings = context->getSettingsRef(); @@ -712,40 +657,27 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (options.only_analyze) { - if constexpr (pipeline_with_processors) - pipeline.init(Pipe(std::make_shared(source_header))); - else - pipeline.streams.emplace_back(std::make_shared(source_header)); + pipeline.init(Pipe(std::make_shared(source_header))); if (expressions.prewhere_info) { - if constexpr (pipeline_with_processors) - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - expressions.prewhere_info->prewhere_actions, - expressions.prewhere_info->prewhere_column_name, - expressions.prewhere_info->remove_prewhere_column); - }); - else - pipeline.streams.back() = std::make_shared( - pipeline.streams.back(), expressions.prewhere_info->prewhere_actions, - expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + expressions.prewhere_info->prewhere_actions, + expressions.prewhere_info->prewhere_column_name, + expressions.prewhere_info->remove_prewhere_column); + }); // To remove additional columns in dry run // For example, sample column which can be removed in this stage if (expressions.prewhere_info->remove_columns_actions) { - if constexpr (pipeline_with_processors) + pipeline.addSimpleTransform([&](const Block & header) { - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, expressions.prewhere_info->remove_columns_actions); - }); - } - else - pipeline.streams.back() = std::make_shared(pipeline.streams.back(), expressions.prewhere_info->remove_columns_actions); + return std::make_shared(header, expressions.prewhere_info->remove_columns_actions); + }); } } } @@ -753,17 +685,11 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS { if (prepared_input) { - if constexpr (pipeline_with_processors) - pipeline.init(Pipe(std::make_shared(prepared_input))); - else - pipeline.streams.push_back(prepared_input); + pipeline.init(Pipe(std::make_shared(prepared_input))); } else if (prepared_pipe) { - if constexpr (pipeline_with_processors) - pipeline.init(std::move(*prepared_pipe)); - else - pipeline.streams.push_back(std::make_shared(std::move(*prepared_pipe))); + pipeline.init(std::move(*prepared_pipe)); } if (from_stage == QueryProcessingStage::WithMergeableState && @@ -774,7 +700,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE); /** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */ - executeFetchColumns(from_stage, pipeline, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere, save_context_and_storage); + executeFetchColumns(from_stage, pipeline, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere); LOG_TRACE(log, "{} -> {}", QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage)); } @@ -817,12 +743,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS } if (query.limitLength()) - { - if constexpr (pipeline_with_processors) - executePreLimit(pipeline, true); - else - executePreLimit(pipeline); - } + executePreLimit(pipeline, true); } }; @@ -840,31 +761,17 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS { if (expressions.hasFilter()) { - if constexpr (pipeline_with_processors) + pipeline.addSimpleTransform([&](const Block & block, QueryPipeline::StreamType stream_type) -> ProcessorPtr { - pipeline.addSimpleTransform([&](const Block & block, QueryPipeline::StreamType stream_type) -> ProcessorPtr - { - bool on_totals = stream_type == QueryPipeline::StreamType::Totals; + bool on_totals = stream_type == QueryPipeline::StreamType::Totals; - return std::make_shared( - block, - expressions.filter_info->actions, - expressions.filter_info->column_name, - expressions.filter_info->do_remove_column, - on_totals); - }); - } - else - { - pipeline.transform([&](auto & stream) - { - stream = std::make_shared( - stream, - expressions.filter_info->actions, - expressions.filter_info->column_name, - expressions.filter_info->do_remove_column); - }); - } + return std::make_shared( + block, + expressions.filter_info->actions, + expressions.filter_info->column_name, + expressions.filter_info->do_remove_column, + on_totals); + }); } if (expressions.hasJoin()) @@ -872,59 +779,43 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS Block join_result_sample; JoinPtr join = expressions.before_join->getTableJoinAlgo(); - if constexpr (pipeline_with_processors) + join_result_sample = ExpressionBlockInputStream( + std::make_shared(pipeline.getHeader()), expressions.before_join).getHeader(); + + /// In case joined subquery has totals, and we don't, add default chunk to totals. + bool default_totals = false; + if (!pipeline.hasTotals()) { - join_result_sample = ExpressionBlockInputStream( - std::make_shared(pipeline.getHeader()), expressions.before_join).getHeader(); - - /// In case joined subquery has totals, and we don't, add default chunk to totals. - bool default_totals = false; - if (!pipeline.hasTotals()) - { - pipeline.addDefaultTotals(); - default_totals = true; - } - - bool inflating_join = false; - if (join) - { - inflating_join = true; - if (auto * hash_join = typeid_cast(join.get())) - inflating_join = isCross(hash_join->getKind()); - } - - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType type) - { - bool on_totals = type == QueryPipeline::StreamType::Totals; - std::shared_ptr ret; - if (inflating_join) - ret = std::make_shared(header, expressions.before_join, on_totals, default_totals); - else - ret = std::make_shared(header, expressions.before_join, on_totals, default_totals); - - return ret; - }); + pipeline.addDefaultTotals(); + default_totals = true; } - else + + bool inflating_join = false; + if (join) { - /// Applies to all sources except stream_with_non_joined_data. - for (auto & stream : pipeline.streams) - stream = std::make_shared(stream, expressions.before_join); - - join_result_sample = pipeline.firstStream()->getHeader(); + inflating_join = true; + if (auto * hash_join = typeid_cast(join.get())) + inflating_join = isCross(hash_join->getKind()); } + pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType type) + { + bool on_totals = type == QueryPipeline::StreamType::Totals; + std::shared_ptr ret; + if (inflating_join) + ret = std::make_shared(header, expressions.before_join, on_totals, default_totals); + else + ret = std::make_shared(header, expressions.before_join, on_totals, default_totals); + + return ret; + }); + if (join) { if (auto stream = join->createStreamWithNonJoinedRows(join_result_sample, settings.max_block_size)) { - if constexpr (pipeline_with_processors) - { - auto source = std::make_shared(std::move(stream)); - pipeline.addDelayedStream(source); - } - else - pipeline.stream_with_non_joined_data = std::move(stream); + auto source = std::make_shared(std::move(stream)); + pipeline.addDelayedStream(source); } } } @@ -1009,28 +900,14 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (query.limitLength() && !query.limit_with_ties && pipeline.hasMoreThanOneStream() && !query.distinct && !expressions.hasLimitBy() && !settings.extremes) { - if constexpr (pipeline_with_processors) - executePreLimit(pipeline, false); - else - executePreLimit(pipeline); - + executePreLimit(pipeline, false); has_prelimit = true; } - bool need_merge_streams = need_second_distinct_pass || query.limitBy() - || (!pipeline_with_processors && query.limitLength()); /// Don't merge streams for pre-limit more. - - if constexpr (!pipeline_with_processors) - if (pipeline.hasDelayedStream()) - need_merge_streams = true; + bool need_merge_streams = need_second_distinct_pass || query.limitBy(); if (need_merge_streams) - { - if constexpr (pipeline_with_processors) - pipeline.resize(1); - else - executeUnion(pipeline, {}); - } + pipeline.resize(1); /** If there was more than one stream, * then DISTINCT needs to be performed once again after merging all streams. @@ -1054,7 +931,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS */ executeExtremes(pipeline); - if (!(pipeline_with_processors && has_prelimit)) /// Limit is no longer needed if there is prelimit. + if (!has_prelimit) /// Limit is no longer needed if there is prelimit. executeLimit(pipeline); executeOffset(pipeline); @@ -1065,14 +942,10 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executeSubqueriesInSetsAndJoins(pipeline, subqueries_for_sets); } -template void InterpreterSelectQuery::executeFetchColumns( - QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, - const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere, - QueryPipeline & save_context_and_storage) + QueryProcessingStage::Enum processing_stage, QueryPipeline & pipeline, + const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere) { - constexpr bool pipeline_with_processors = std::is_same::value; - auto & query = getSelectQuery(); const Settings & settings = context->getSettingsRef(); @@ -1123,13 +996,13 @@ void InterpreterSelectQuery::executeFetchColumns( argument_types[j] = header.getByName(desc->argument_names[j]).type; Block block_with_count{ - {std::move(column), std::make_shared(func, argument_types, desc->parameters), desc->column_name}}; + {nullptr, std::make_shared(func, argument_types, desc->parameters), desc->column_name}}; - auto istream = std::make_shared(block_with_count); - if constexpr (pipeline_with_processors) - pipeline.init(Pipe(std::make_shared(istream))); - else - pipeline.streams.emplace_back(istream); + Chunk chunk(Columns(), column->size()); + chunk.addColumn(std::move(column)); + + auto source = std::make_shared(std::move(block_with_count), std::move(chunk)); + pipeline.init(Pipe(std::move(source))); from_stage = QueryProcessingStage::WithMergeableState; analysis_result.first_stage = false; return; @@ -1266,7 +1139,7 @@ void InterpreterSelectQuery::executeFetchColumns( /// Remove columns which will be added by prewhere. required_columns.erase(std::remove_if(required_columns.begin(), required_columns.end(), [&](const String & name) { - return !!required_columns_after_prewhere_set.count(name); + return required_columns_after_prewhere_set.count(name) != 0; }), required_columns.end()); if (prewhere_info) @@ -1377,11 +1250,7 @@ void InterpreterSelectQuery::executeFetchColumns( interpreter_subquery->ignoreWithTotals(); } - if constexpr (pipeline_with_processors) - /// Just use pipeline from subquery. - pipeline = interpreter_subquery->executeWithProcessors(); - else - pipeline.streams = interpreter_subquery->executeWithMultipleStreams(save_context_and_storage); + pipeline = interpreter_subquery->execute().pipeline; } else if (storage) { @@ -1411,48 +1280,9 @@ void InterpreterSelectQuery::executeFetchColumns( query_info.input_sorting_info = query_info.order_by_optimizer->getInputOrder(storage); } + Pipes pipes = storage->read(required_columns, query_info, *context, processing_stage, max_block_size, max_streams); - BlockInputStreams streams; - Pipes pipes; - - if (pipeline_with_processors) - pipes = storage->read(required_columns, query_info, *context, processing_stage, max_block_size, max_streams); - else - streams = storage->readStreams(required_columns, query_info, *context, processing_stage, max_block_size, max_streams); - - if (streams.empty() && !pipeline_with_processors) - { - streams = {std::make_shared(storage->getSampleBlockForColumns(required_columns))}; - - if (query_info.prewhere_info) - { - if (query_info.prewhere_info->alias_actions) - { - streams.back() = std::make_shared( - streams.back(), - query_info.prewhere_info->alias_actions); - } - - streams.back() = std::make_shared( - streams.back(), - prewhere_info->prewhere_actions, - prewhere_info->prewhere_column_name, - prewhere_info->remove_prewhere_column); - - // To remove additional columns - // In some cases, we did not read any marks so that the pipeline.streams is empty - // Thus, some columns in prewhere are not removed as expected - // This leads to mismatched header in distributed table - if (query_info.prewhere_info->remove_columns_actions) - { - streams.back() = std::make_shared(streams.back(), query_info.prewhere_info->remove_columns_actions); - } - } - } - - /// Copy-paste from prev if. - /// Code is temporarily copy-pasted while moving to new pipeline. - if (pipes.empty() && pipeline_with_processors) + if (pipes.empty()) { Pipe pipe(std::make_shared(storage->getSampleBlockForColumns(required_columns))); @@ -1468,6 +1298,10 @@ void InterpreterSelectQuery::executeFetchColumns( prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column)); + // To remove additional columns + // In some cases, we did not read any marks so that the pipeline.streams is empty + // Thus, some columns in prewhere are not removed as expected + // This leads to mismatched header in distributed table if (query_info.prewhere_info->remove_columns_actions) pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), query_info.prewhere_info->remove_columns_actions)); } @@ -1475,14 +1309,8 @@ void InterpreterSelectQuery::executeFetchColumns( pipes.emplace_back(std::move(pipe)); } - for (auto & stream : streams) - stream->addTableLock(table_lock); - - if constexpr (pipeline_with_processors) - { - /// Table lock is stored inside pipeline here. - pipeline.addTableLock(table_lock); - } + /// Table lock is stored inside pipeline here. + pipeline.addTableLock(table_lock); /// Set the limits and quota for reading data, the speed and time of the query. { @@ -1510,16 +1338,6 @@ void InterpreterSelectQuery::executeFetchColumns( auto quota = context->getQuota(); - for (auto & stream : streams) - { - if (!options.ignore_limits) - stream->setLimits(limits); - - if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete)) - stream->setQuota(quota); - } - - /// Copy-paste for (auto & pipe : pipes) { if (!options.ignore_limits) @@ -1530,47 +1348,13 @@ void InterpreterSelectQuery::executeFetchColumns( } } - if constexpr (pipeline_with_processors) - { - if (streams.size() == 1 || pipes.size() == 1) - pipeline.setMaxThreads(1); + if (pipes.size() == 1) + pipeline.setMaxThreads(1); - /// Unify streams. They must have same headers. - if (streams.size() > 1) - { - /// Unify streams in case they have different headers. - auto first_header = streams.at(0)->getHeader(); + for (auto & pipe : pipes) + pipe.enableQuota(); - if (first_header.columns() > 1 && first_header.has("_dummy")) - first_header.erase("_dummy"); - - for (auto & stream : streams) - { - auto header = stream->getHeader(); - auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; - if (!blocksHaveEqualStructure(first_header, header)) - stream = std::make_shared(stream, first_header, mode); - } - } - - for (auto & stream : streams) - { - bool force_add_agg_info = processing_stage == QueryProcessingStage::WithMergeableState; - auto source = std::make_shared(stream, force_add_agg_info); - - if (processing_stage == QueryProcessingStage::Complete) - source->addTotalsPort(); - - pipes.emplace_back(std::move(source)); - } - - for (auto & pipe : pipes) - pipe.enableQuota(); - - pipeline.init(std::move(pipes)); - } - else - pipeline.streams = std::move(streams); + pipeline.init(std::move(pipes)); } else throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR); @@ -1578,32 +1362,14 @@ void InterpreterSelectQuery::executeFetchColumns( /// Aliases in table declaration. if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions) { - if constexpr (pipeline_with_processors) + pipeline.addSimpleTransform([&](const Block & header) { - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, alias_actions); - }); - } - else - { - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, alias_actions); - }); - } + return std::make_shared(header, alias_actions); + }); } } -void InterpreterSelectQuery::executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter) -{ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, expression, getSelectQuery().where()->getColumnName(), remove_filter); - }); -} - void InterpreterSelectQuery::executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter) { pipeline.addSimpleTransform([&](const Block & block, QueryPipeline::StreamType stream_type) @@ -1613,69 +1379,6 @@ void InterpreterSelectQuery::executeWhere(QueryPipeline & pipeline, const Expres }); } -void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final) -{ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, expression); - }); - - Block header = pipeline.firstStream()->getHeader(); - ColumnNumbers keys; - for (const auto & key : query_analyzer->aggregationKeys()) - keys.push_back(header.getPositionByName(key.name)); - - AggregateDescriptions aggregates = query_analyzer->aggregates(); - for (auto & descr : aggregates) - if (descr.arguments.empty()) - for (const auto & name : descr.argument_names) - descr.arguments.push_back(header.getPositionByName(name)); - - const Settings & settings = context->getSettingsRef(); - - /** Two-level aggregation is useful in two cases: - * 1. Parallel aggregation is done, and the results should be merged in parallel. - * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. - */ - bool allow_to_use_two_level_group_by = pipeline.streams.size() > 1 || settings.max_bytes_before_external_group_by != 0; - - Aggregator::Params params(header, keys, aggregates, - overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), - allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), - settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); - - /// If there are several sources, then we perform parallel aggregation - if (pipeline.streams.size() > 1) - { - pipeline.firstStream() = std::make_shared( - pipeline.streams, pipeline.stream_with_non_joined_data, params, final, - max_streams, - settings.aggregation_memory_efficient_merge_threads - ? static_cast(settings.aggregation_memory_efficient_merge_threads) - : static_cast(settings.max_threads)); - - pipeline.stream_with_non_joined_data = nullptr; - pipeline.streams.resize(1); - } - else - { - BlockInputStreams inputs; - if (!pipeline.streams.empty()) - inputs.push_back(pipeline.firstStream()); - else - pipeline.streams.resize(1); - - if (pipeline.stream_with_non_joined_data) - inputs.push_back(pipeline.stream_with_non_joined_data); - - pipeline.firstStream() = std::make_shared(std::make_shared(inputs), params, final); - - pipeline.stream_with_non_joined_data = nullptr; - } -} - void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final) { @@ -1749,53 +1452,6 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const } -void InterpreterSelectQuery::executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final) -{ - Block header = pipeline.firstStream()->getHeader(); - - ColumnNumbers keys; - for (const auto & key : query_analyzer->aggregationKeys()) - keys.push_back(header.getPositionByName(key.name)); - - /** There are two modes of distributed aggregation. - * - * 1. In different threads read from the remote servers blocks. - * Save all the blocks in the RAM. Merge blocks. - * If the aggregation is two-level - parallelize to the number of buckets. - * - * 2. In one thread, read blocks from different servers in order. - * RAM stores only one block from each server. - * If the aggregation is a two-level aggregation, we consistently merge the blocks of each next level. - * - * The second option consumes less memory (up to 256 times less) - * in the case of two-level aggregation, which is used for large results after GROUP BY, - * but it can work more slowly. - */ - - const Settings & settings = context->getSettingsRef(); - - Aggregator::Params params(header, keys, query_analyzer->aggregates(), overflow_row, settings.max_threads); - - if (!settings.distributed_aggregation_memory_efficient) - { - /// We union several sources into one, parallelizing the work. - executeUnion(pipeline, {}); - - /// Now merge the aggregated blocks - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params, final, settings.max_threads); - } - else - { - pipeline.firstStream() = std::make_shared(pipeline.streams, params, final, - max_streams, - settings.aggregation_memory_efficient_merge_threads - ? static_cast(settings.aggregation_memory_efficient_merge_threads) - : static_cast(settings.max_threads)); - - pipeline.streams.resize(1); - } -} - void InterpreterSelectQuery::executeMergeAggregated(QueryPipeline & pipeline, bool overflow_row, bool final) { Block header_before_merge = pipeline.getHeader(); @@ -1856,14 +1512,6 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPipeline & pipeline, bo } -void InterpreterSelectQuery::executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression) -{ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, expression, getSelectQuery().having()->getColumnName()); - }); -} - void InterpreterSelectQuery::executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression) { pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr @@ -1876,22 +1524,6 @@ void InterpreterSelectQuery::executeHaving(QueryPipeline & pipeline, const Expre } -void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final) -{ - executeUnion(pipeline, {}); - - const Settings & settings = context->getSettingsRef(); - - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), - overflow_row, - expression, - has_having ? getSelectQuery().having()->getColumnName() : "", - settings.totals_mode, - settings.totals_auto_threshold, - final); -} - void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final) { const Settings & settings = context->getSettingsRef(); @@ -1905,31 +1537,6 @@ void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bo } -void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificator modificator) -{ - executeUnion(pipeline, {}); - - Block header = pipeline.firstStream()->getHeader(); - - ColumnNumbers keys; - - for (const auto & key : query_analyzer->aggregationKeys()) - keys.push_back(header.getPositionByName(key.name)); - - const Settings & settings = context->getSettingsRef(); - - Aggregator::Params params(header, keys, query_analyzer->aggregates(), - false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, - SettingUInt64(0), SettingUInt64(0), - settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); - - if (modificator == Modificator::ROLLUP) - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); - else - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); -} - void InterpreterSelectQuery::executeRollupOrCube(QueryPipeline & pipeline, Modificator modificator) { pipeline.resize(1); @@ -1964,14 +1571,6 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPipeline & pipeline, Modif } -void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression) -{ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, expression); - }); -} - void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression) { pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr @@ -1980,68 +1579,6 @@ void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const E }); } -void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoPtr input_sorting_info) -{ - auto & query = getSelectQuery(); - SortDescription output_order_descr = getSortDescription(query, *context); - const Settings & settings = context->getSettingsRef(); - UInt64 limit = getLimitForSorting(query, *context); - - if (input_sorting_info) - { - /* Case of sorting with optimization using sorting key. - * We have several threads, each of them reads batch of parts in direct - * or reverse order of sorting key using one input stream per part - * and then merge them into one sorted stream. - * At this stage we merge per-thread streams into one. - * If the input is sorted by some prefix of the sorting key required for output, - * we have to finish sorting after the merge. - */ - - bool need_finish_sorting = (input_sorting_info->order_key_prefix_descr.size() < output_order_descr.size()); - - UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); - executeMergeSorted(pipeline, input_sorting_info->order_key_prefix_descr, limit_for_merging); - - if (need_finish_sorting) - { - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, output_order_descr, limit); - }); - - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), input_sorting_info->order_key_prefix_descr, - output_order_descr, settings.max_block_size, limit); - } - } - else - { - pipeline.transform([&](auto & stream) - { - auto sorting_stream = std::make_shared(stream, output_order_descr, limit); - - /// Limits on sorting - IBlockInputStream::LocalLimits limits; - limits.mode = IBlockInputStream::LIMITS_TOTAL; - limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sorting_stream->setLimits(limits); - - auto merging_stream = std::make_shared( - sorting_stream, output_order_descr, settings.max_block_size, limit, - settings.max_bytes_before_remerge_sort, - settings.max_bytes_before_external_sort / pipeline.streams.size(), - context->getTemporaryVolume(), - settings.temporary_files_codec, - settings.min_free_disk_space_for_temporary_data); - - stream = merging_stream; - }); - - /// If there are several streams, we merge them into one - executeMergeSorted(pipeline, output_order_descr, limit); - } -} void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr input_sorting_info) { @@ -2138,41 +1675,6 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting } -void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) -{ - auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, *context); - UInt64 limit = getLimitForSorting(query, *context); - - /// If there are several streams, then we merge them into one - if (pipeline.hasMoreThanOneStream()) - { - unifyStreams(pipeline, pipeline.firstStream()->getHeader()); - executeMergeSorted(pipeline, order_descr, limit); - } -} - - -void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline, const SortDescription & sort_description, UInt64 limit) -{ - if (pipeline.hasMoreThanOneStream()) - { - const Settings & settings = context->getSettingsRef(); - - /** MergingSortedBlockInputStream reads the sources sequentially. - * To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream. - */ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream); - }); - - pipeline.firstStream() = std::make_shared( - pipeline.streams, sort_description, settings.max_block_size, limit); - pipeline.streams.resize(1); - } -} - void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline) { auto & query = getSelectQuery(); @@ -2202,14 +1704,6 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline, const } -void InterpreterSelectQuery::executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression) -{ - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, expression); - }); -} - void InterpreterSelectQuery::executeProjection(QueryPipeline & pipeline, const ExpressionActionsPtr & expression) { pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr @@ -2219,28 +1713,6 @@ void InterpreterSelectQuery::executeProjection(QueryPipeline & pipeline, const E } -void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_order, Names columns) -{ - auto & query = getSelectQuery(); - if (query.distinct) - { - const Settings & settings = context->getSettingsRef(); - - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context); - UInt64 limit_for_distinct = 0; - - /// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows. - if ((!query.orderBy() || !before_order) && !query.limit_with_ties) - limit_for_distinct = limit_length + limit_offset; - - pipeline.transform([&](auto & stream) - { - SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode); - stream = std::make_shared(stream, limits, limit_for_distinct, columns); - }); - } -} - void InterpreterSelectQuery::executeDistinct(QueryPipeline & pipeline, bool before_order, Names columns) { auto & query = getSelectQuery(); @@ -2268,51 +1740,6 @@ void InterpreterSelectQuery::executeDistinct(QueryPipeline & pipeline, bool befo } -void InterpreterSelectQuery::executeUnion(Pipeline & pipeline, Block header) -{ - /// If there are still several streams, then we combine them into one - if (pipeline.hasMoreThanOneStream()) - { - if (!header) - header = pipeline.firstStream()->getHeader(); - - unifyStreams(pipeline, std::move(header)); - - pipeline.firstStream() = std::make_shared(pipeline.streams, pipeline.stream_with_non_joined_data, max_streams); - pipeline.stream_with_non_joined_data = nullptr; - pipeline.streams.resize(1); - pipeline.union_stream = true; - } - else if (pipeline.stream_with_non_joined_data) - { - pipeline.streams.push_back(pipeline.stream_with_non_joined_data); - pipeline.stream_with_non_joined_data = nullptr; - } -} - - -/// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined. -void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) -{ - auto & query = getSelectQuery(); - /// If there is LIMIT - if (query.limitLength()) - { - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context); - SortDescription sort_descr; - if (query.limit_with_ties) - { - if (!query.orderBy()) - throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - sort_descr = getSortDescription(query, *context); - } - pipeline.transform([&, limit = limit_length + limit_offset](auto & stream) - { - stream = std::make_shared(stream, limit, 0, false, false, query.limit_with_ties, sort_descr); - }); - } -} - /// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined. void InterpreterSelectQuery::executePreLimit(QueryPipeline & pipeline, bool do_not_skip_offset) { @@ -2334,24 +1761,6 @@ void InterpreterSelectQuery::executePreLimit(QueryPipeline & pipeline, bool do_n } -void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline) -{ - auto & query = getSelectQuery(); - if (!query.limitByLength() || !query.limitBy()) - return; - - Names columns; - for (const auto & elem : query.limitBy()->children) - columns.emplace_back(elem->getColumnName()); - UInt64 length = getLimitUIntValue(query.limitByLength(), *context, "LIMIT"); - UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context, "OFFSET") : 0); - - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, length, offset, columns); - }); -} - void InterpreterSelectQuery::executeLimitBy(QueryPipeline & pipeline) { auto & query = getSelectQuery(); @@ -2400,73 +1809,6 @@ namespace } } -void InterpreterSelectQuery::executeLimit(Pipeline & pipeline) -{ - auto & query = getSelectQuery(); - /// If there is LIMIT - if (query.limitLength()) - { - /** Rare case: - * if there is no WITH TOTALS and there is a subquery in FROM, and there is WITH TOTALS on one of the levels, - * then when using LIMIT, you should read the data to the end, rather than cancel the query earlier, - * because if you cancel the query, we will not get `totals` data from the remote server. - * - * Another case: - * if there is WITH TOTALS and there is no ORDER BY, then read the data to the end, - * otherwise TOTALS is counted according to incomplete data. - */ - bool always_read_till_end = false; - - if (query.group_by_with_totals && !query.orderBy()) - always_read_till_end = true; - - if (!query.group_by_with_totals && hasWithTotalsInAnySubqueryInFromClause(query)) - always_read_till_end = true; - - SortDescription order_descr; - if (query.limit_with_ties) - { - if (!query.orderBy()) - throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); - order_descr = getSortDescription(query, *context); - } - - UInt64 limit_length; - UInt64 limit_offset; - std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context); - - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, limit_length, limit_offset, always_read_till_end, false, query.limit_with_ties, order_descr); - }); - } -} - -void InterpreterSelectQuery::executeOffset(Pipeline & /* pipeline */) {} - -void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline) -{ - auto & query = getSelectQuery(); - if (query.orderBy()) - { - SortDescription order_descr = getSortDescription(query, *context); - SortDescription fill_descr; - for (auto & desc : order_descr) - { - if (desc.with_fill) - fill_descr.push_back(desc); - } - - if (fill_descr.empty()) - return; - - pipeline.transform([&](auto & stream) - { - stream = std::make_shared(stream, fill_descr); - }); - } -} - void InterpreterSelectQuery::executeWithFill(QueryPipeline & pipeline) { auto & query = getSelectQuery(); @@ -2557,18 +1899,6 @@ void InterpreterSelectQuery::executeOffset(QueryPipeline & pipeline) } } - -void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline) -{ - if (!context->getSettingsRef().extremes) - return; - - pipeline.transform([&](auto & stream) - { - stream->enableExtremes(); - }); -} - void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline) { if (!context->getSettingsRef().extremes) @@ -2577,23 +1907,6 @@ void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline) pipeline.addExtremesTransform(); } - -void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline, const SubqueriesForSets & subqueries_for_sets) -{ - /// Merge streams to one. Use MergeSorting if data was read in sorted order, Union otherwise. - if (query_info.input_sorting_info) - { - if (pipeline.stream_with_non_joined_data) - throw Exception("Using read in order optimization, but has stream with non-joined data in pipeline", ErrorCodes::LOGICAL_ERROR); - executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0); - } - else - executeUnion(pipeline, {}); - - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), subqueries_for_sets, *context); -} - void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, const SubqueriesForSets & subqueries_for_sets) { if (query_info.input_sorting_info) @@ -2610,25 +1923,6 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pip } -void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header) -{ - /// Unify streams in case they have different headers. - - /// TODO: remove previous addition of _dummy column. - if (header.columns() > 1 && header.has("_dummy")) - header.erase("_dummy"); - - for (auto & stream : pipeline.streams) - { - auto stream_header = stream->getHeader(); - auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; - - if (!blocksHaveEqualStructure(header, stream_header)) - stream = std::make_shared(stream, header, mode); - } -} - - void InterpreterSelectQuery::ignoreWithTotals() { getSelectQuery().group_by_with_totals = false; diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index b97ff65e988..ca7fb4c72ba 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -77,12 +77,6 @@ public: /// Execute a query. Get the stream of blocks to read. BlockIO execute() override; - /// Execute the query and return multuple streams for parallel processing. - BlockInputStreams executeWithMultipleStreams(QueryPipeline & parent_pipeline); - - QueryPipeline executeWithProcessors() override; - bool canExecuteWithProcessors() const override { return true; } - bool ignoreLimits() const override { return options.ignore_limits; } bool ignoreQuota() const override { return options.ignore_quota; } @@ -108,89 +102,15 @@ private: Block getSampleBlockImpl(); - struct Pipeline - { - /** Streams of data. - * The source data streams are produced in the executeFetchColumns function. - * Then they are converted (wrapped in other streams) using the `execute*` functions, - * to get the whole pipeline running the query. - */ - BlockInputStreams streams; - - /** When executing FULL or RIGHT JOIN, there will be a data stream from which you can read "not joined" rows. - * It has a special meaning, since reading from it should be done after reading from the main streams. - * It is appended to the main streams in UnionBlockInputStream or ParallelAggregatingBlockInputStream. - */ - BlockInputStreamPtr stream_with_non_joined_data; - bool union_stream = false; - - /// Cache value of InterpreterSelectQuery::max_streams - size_t max_threads = 1; - - BlockInputStreamPtr & firstStream() { return streams.at(0); } - - template - void transform(Transform && transformation) - { - for (auto & stream : streams) - transformation(stream); - - if (stream_with_non_joined_data) - transformation(stream_with_non_joined_data); - } - - bool hasMoreThanOneStream() const - { - return streams.size() + (stream_with_non_joined_data ? 1 : 0) > 1; - } - - /// Resulting stream is mix of other streams data. Distinct and/or order guaranties are broken. - bool hasMixedStreams() const - { - return hasMoreThanOneStream() || union_stream; - } - - bool hasDelayedStream() const { return stream_with_non_joined_data != nullptr; } - bool initialized() const { return !streams.empty(); } - - /// Compatibility with QueryPipeline (Processors) - void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; } - size_t getNumThreads() const { return max_threads; } - }; - - template - void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe, QueryPipeline & save_context_and_storage); + void executeImpl(QueryPipeline & pipeline, const BlockInputStreamPtr & prepared_input, std::optional prepared_pipe); /// Different stages of query execution. - /// dry_run - don't read from table, use empty header block instead. - void executeWithMultipleStreamsImpl(Pipeline & pipeline, const BlockInputStreamPtr & input, bool dry_run); - - template - void executeFetchColumns(QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, + void executeFetchColumns( + QueryProcessingStage::Enum processing_stage, + QueryPipeline & pipeline, const PrewhereInfoPtr & prewhere_info, - const Names & columns_to_remove_after_prewhere, - QueryPipeline & save_context_and_storage); - - void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); - void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final); - void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final); - void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); - void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); - static void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression); - void executeOrder(Pipeline & pipeline, InputSortingInfoPtr sorting_info); - void executeWithFill(Pipeline & pipeline); - void executeMergeSorted(Pipeline & pipeline); - void executePreLimit(Pipeline & pipeline); - void executeUnion(Pipeline & pipeline, Block header); - void executeLimitBy(Pipeline & pipeline); - void executeLimit(Pipeline & pipeline); - void executeOffset(Pipeline & pipeline); - static void executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression); - void executeDistinct(Pipeline & pipeline, bool before_order, Names columns); - void executeExtremes(Pipeline & pipeline); - void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, const std::unordered_map & subqueries_for_sets); - void executeMergeSorted(Pipeline & pipeline, const SortDescription & sort_description, UInt64 limit); + const Names & columns_to_remove_after_prewhere); void executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final); @@ -213,17 +133,12 @@ private: String generateFilterActions(ExpressionActionsPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const; - /// Add ConvertingBlockInputStream to specified header. - static void unifyStreams(Pipeline & pipeline, Block header); - enum class Modificator { ROLLUP = 0, CUBE = 1 }; - void executeRollupOrCube(Pipeline & pipeline, Modificator modificator); - void executeRollupOrCube(QueryPipeline & pipeline, Modificator modificator); /** If there is a SETTINGS section in the SELECT query, then apply settings from it. diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 7fe124b31e6..7b86616555a 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -3,15 +3,9 @@ #include #include #include -#include -#include -#include -#include #include -#include #include #include -#include #include #include @@ -180,69 +174,10 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock( } -BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams(QueryPipeline & parent_pipeline) -{ - BlockInputStreams nested_streams; - - for (auto & interpreter : nested_interpreters) - { - BlockInputStreams streams = interpreter->executeWithMultipleStreams(parent_pipeline); - nested_streams.insert(nested_streams.end(), streams.begin(), streams.end()); - } - - /// Unify data structure. - if (nested_interpreters.size() > 1) - { - for (auto & stream : nested_streams) - stream = std::make_shared(stream, result_header,ConvertingBlockInputStream::MatchColumnsMode::Position); - parent_pipeline.addInterpreterContext(context); - } - - /// Update max_streams due to: - /// - max_distributed_connections for Distributed() engine - /// - max_streams_to_max_threads_ratio - /// - /// XXX: res.pipeline.getMaxThreads() cannot be used since it is capped to - /// number of streams, which is empty for non-Processors case. - max_streams = (*std::min_element(nested_interpreters.begin(), nested_interpreters.end(), [](const auto &a, const auto &b) - { - return a->getMaxStreams() < b->getMaxStreams(); - }))->getMaxStreams(); - - return nested_streams; -} - - BlockIO InterpreterSelectWithUnionQuery::execute() { BlockIO res; - BlockInputStreams nested_streams = executeWithMultipleStreams(res.pipeline); - BlockInputStreamPtr result_stream; - - if (nested_streams.empty()) - { - result_stream = std::make_shared(getSampleBlock()); - } - else if (nested_streams.size() == 1) - { - result_stream = nested_streams.front(); - nested_streams.clear(); - } - else - { - result_stream = std::make_shared(nested_streams, nullptr, max_streams); - nested_streams.clear(); - } - - res.in = result_stream; - res.pipeline.addInterpreterContext(context); - return res; -} - - -QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors() -{ - QueryPipeline main_pipeline; + QueryPipeline & main_pipeline = res.pipeline; std::vector pipelines; bool has_main_pipeline = false; @@ -254,12 +189,12 @@ QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors() if (!has_main_pipeline) { has_main_pipeline = true; - main_pipeline = interpreter->executeWithProcessors(); + main_pipeline = interpreter->execute().pipeline; headers.emplace_back(main_pipeline.getHeader()); } else { - pipelines.emplace_back(interpreter->executeWithProcessors()); + pipelines.emplace_back(interpreter->execute().pipeline); headers.emplace_back(pipelines.back().getHeader()); } } @@ -280,7 +215,7 @@ QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors() main_pipeline.addInterpreterContext(context); - return main_pipeline; + return res; } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index c7a8e09578b..3b5fe533a84 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -29,12 +29,6 @@ public: BlockIO execute() override; - /// Execute the query without union of streams. - BlockInputStreams executeWithMultipleStreams(QueryPipeline & parent_pipeline); - - QueryPipeline executeWithProcessors() override; - bool canExecuteWithProcessors() const override { return true; } - bool ignoreLimits() const override { return options.ignore_limits; } bool ignoreQuota() const override { return options.ignore_quota; } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 2cc6730b90d..8fc799d0b48 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -280,7 +280,6 @@ static std::tuple executeQueryImpl( /// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion. String query(begin, query_end); BlockIO res; - QueryPipeline & pipeline = res.pipeline; String query_for_logging; @@ -338,7 +337,6 @@ static std::tuple executeQueryImpl( context.resetInputCallbacks(); auto interpreter = InterpreterFactory::get(ast, context, stage); - bool use_processors = interpreter->canExecuteWithProcessors(); std::shared_ptr quota; if (!interpreter->ignoreQuota()) @@ -358,10 +356,9 @@ static std::tuple executeQueryImpl( limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); } - if (use_processors) - pipeline = interpreter->executeWithProcessors(); - else - res = interpreter->execute(); + res = interpreter->execute(); + QueryPipeline & pipeline = res.pipeline; + bool use_processors = pipeline.initialized(); if (res.pipeline.initialized()) use_processors = true; diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 6564fee827e..f9666ed0548 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -397,25 +397,6 @@ void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settin } } -BlockInputStreams IStorage::readStreams( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) -{ - ForceTreeShapedPipeline enable_tree_shape(query_info); - auto pipes = read(column_names, query_info, context, processed_stage, max_block_size, num_streams); - - BlockInputStreams res; - res.reserve(pipes.size()); - - for (auto & pipe : pipes) - res.emplace_back(std::make_shared(std::move(pipe))); - - return res; -} StorageID IStorage::getStorageID() const { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 655ced0d8cf..76a5d72262a 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -303,16 +303,6 @@ public: throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** The same as read, but returns BlockInputStreams. - */ - BlockInputStreams readStreams( - const Names & /*column_names*/, - const SelectQueryInfo & /*query_info*/, - const Context & /*context*/, - QueryProcessingStage::Enum /*processed_stage*/, - size_t /*max_block_size*/, - unsigned /*num_streams*/); - /** Writes the data to a table. * Receives a description of the query, which can contain information about the data write method. * Returns an object by which you can write data sequentially. diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 3d7e26d3817..84cf3a32aa1 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -80,28 +80,6 @@ struct SelectQueryInfo /// Prepared sets are used for indices by storage engine. /// Example: x IN (1, 2, 3) PreparedSets sets; - - /// Temporary flag is needed to support old pipeline with input streams. - /// If enabled, then pipeline returned by storage must be a tree. - /// Processors from the tree can't return ExpandPipeline status. - mutable bool force_tree_shaped_pipeline = false; -}; - -/// RAII class to enable force_tree_shaped_pipeline for SelectQueryInfo. -/// Looks awful, but I hope it's temporary. -struct ForceTreeShapedPipeline -{ - explicit ForceTreeShapedPipeline(const SelectQueryInfo & info_) : info(info_) - { - force_tree_shaped_pipeline = info.force_tree_shaped_pipeline; - info.force_tree_shaped_pipeline = true; - } - - ~ForceTreeShapedPipeline() { info.force_tree_shaped_pipeline = force_tree_shaped_pipeline; } - -private: - bool force_tree_shaped_pipeline; - const SelectQueryInfo & info; }; } diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 636c7f9d64d..06c0bcacb4c 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -64,16 +64,8 @@ Pipes StorageView::read( QueryPipeline pipeline; InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, {}, column_names); - /// FIXME res may implicitly use some objects owned be pipeline, but them will be destructed after return - if (query_info.force_tree_shaped_pipeline) - { - BlockInputStreams streams = interpreter.executeWithMultipleStreams(pipeline); - for (auto & stream : streams) - pipes.emplace_back(std::make_shared(std::move(stream))); - } - else - /// TODO: support multiple streams here. Need more general interface than pipes. - pipes.emplace_back(interpreter.executeWithProcessors().getPipe()); + /// TODO: support multiple streams here. Need more general interface than pipes. + pipes.emplace_back(interpreter.execute().pipeline.getPipe()); /// It's expected that the columns read from storage are not constant. /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. From 5572f6c27666a0bc5cb11997d472c317da8dfb68 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 12:07:40 +0300 Subject: [PATCH 032/150] Fix style. --- src/Interpreters/IInterpreter.h | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Interpreters/IInterpreter.h b/src/Interpreters/IInterpreter.h index 63f36f1b18a..41613d30e87 100644 --- a/src/Interpreters/IInterpreter.h +++ b/src/Interpreters/IInterpreter.h @@ -2,14 +2,8 @@ #include -#include - namespace DB { -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} /** Interpreters interface for different queries. */ @@ -25,7 +19,7 @@ public: virtual bool ignoreQuota() const { return false; } virtual bool ignoreLimits() const { return false; } - virtual ~IInterpreter() {} + virtual ~IInterpreter() = default; }; } From 29a580434233c0e39a7811042e1280871d9b0b04 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 28 May 2020 12:46:31 +0300 Subject: [PATCH 033/150] S3 Poco HTTP Client WIP (rewind buffer). --- src/IO/S3/PocoHttpClient.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/IO/S3/PocoHttpClient.cpp b/src/IO/S3/PocoHttpClient.cpp index cad100ab717..9e2ef40fe50 100644 --- a/src/IO/S3/PocoHttpClient.cpp +++ b/src/IO/S3/PocoHttpClient.cpp @@ -96,6 +96,11 @@ void PocoHttpClient::MakeRequestInternal( { if (request.GetContentBody()) { + if (attempt > 0) /// rewind content body buffer. + { + request.GetContentBody()->clear(); + request.GetContentBody()->seekg(0); + } auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); LOG_DEBUG( &Logger::get("AWSClient"), "Written {} bytes to request body", size); From 6efa4d1d4f3457cb5cc90832ebb9e8c90c9cab0c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 13:41:59 +0300 Subject: [PATCH 034/150] Fix style. --- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b6efa5d6d46..300ea63af84 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -204,7 +204,7 @@ BlockIO InterpreterInsertQuery::execute() { /// Passing 1 as subquery_depth will disable limiting size of intermediate result. InterpreterSelectWithUnionQuery interpreter_select{ query.select, context, SelectQueryOptions(QueryProcessingStage::Complete, 1)}; - res.pipeline = interpreter_select.executeWithProcessors(); + res = interpreter_select.execute(); if (table->supportsParallelInsert() && settings.max_insert_threads > 1) out_streams_size = std::min(size_t(settings.max_insert_threads), res.pipeline.getNumStreams()); From da0052858dcaa1171d1e8af0635914d6a1764eb0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 13:57:04 +0300 Subject: [PATCH 035/150] Fix build. --- .../ClusterProxy/SelectStreamFactory.cpp | 38 +++++-------------- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageMerge.cpp | 22 +---------- 4 files changed, 13 insertions(+), 51 deletions(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 53c61bea60e..4b35a982049 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -15,6 +15,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -70,35 +71,14 @@ SelectStreamFactory::SelectStreamFactory( namespace { -Pipe createLocalStream( - const ASTPtr & query_ast, const Block & header, const Context & context, QueryProcessingStage::Enum processed_stage, - bool add_totals_port, bool add_extremes_port, bool force_tree_shaped_pipeline) +QueryPipeline createLocalStream( + const ASTPtr & query_ast, const Block & header, const Context & context, QueryProcessingStage::Enum processed_stage) { checkStackSize(); InterpreterSelectQuery interpreter{query_ast, context, SelectQueryOptions(processed_stage)}; - if (force_tree_shaped_pipeline) - { - /// This flag means that pipeline must be tree-shaped, - /// so we can't enable processors for InterpreterSelectQuery here. - auto stream = interpreter.execute().in; - auto source = std::make_shared(std::move(stream)); - - if (add_totals_port) - source->addTotalsPort(); - if (add_extremes_port) - source->addExtremesPort(); - - Pipe pipe(std::move(source)); - - pipe.addSimpleTransform(std::make_shared( - pipe.getHeader(), header, ConvertingTransform::MatchColumnsMode::Name)); - - return pipe; - } - - auto pipeline = interpreter.executeWithProcessors(); + auto pipeline = interpreter.execute().pipeline; pipeline.addSimpleTransform([&](const Block & source_header) { @@ -116,7 +96,8 @@ Pipe createLocalStream( */ /// return std::make_shared(stream); - return std::move(pipeline).getPipe(); + pipeline.setMaxThreads(1); + return pipeline; } String formattedAST(const ASTPtr & ast) @@ -152,8 +133,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { - res.emplace_back(createLocalStream(modified_query_ast, header, context, processed_stage, - add_totals_port, add_extremes_port, query_info.force_tree_shaped_pipeline)); + res.emplace_back(createLocalStream(modified_query_ast, header, context, processed_stage).getPipe()); }; String modified_query = formattedAST(modified_query_ast); @@ -297,8 +277,8 @@ void SelectStreamFactory::createForShard( } if (try_results.empty() || local_delay < max_remote_delay) - return std::make_shared( - createLocalStream(modified_query_ast, header, context, stage, add_totals_port, add_extremes_port, true)); + return std::make_shared( + createLocalStream(modified_query_ast, header, context, stage)); else { std::vector connections; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f90add64732..24e3e3ac69d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1151,7 +1151,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (num_streams > settings.max_final_threads) num_streams = settings.max_final_threads; - if (num_streams <= 1 || sort_description.empty() || query_info.force_tree_shaped_pipeline) + if (num_streams <= 1 || sort_description.empty()) { Pipe pipe(std::move(pipes), get_merging_processor()); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 95be1275d3d..ac423c001ac 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -234,7 +234,7 @@ Pipes StorageBuffer::read( */ if (processed_stage > QueryProcessingStage::FetchColumns) for (auto & pipe : pipes_from_buffers) - pipe = InterpreterSelectQuery(query_info.query, context, std::move(pipe), SelectQueryOptions(processed_stage)).executeWithProcessors().getPipe(); + pipe = InterpreterSelectQuery(query_info.query, context, std::move(pipe), SelectQueryOptions(processed_stage)).execute().pipeline.getPipe(); if (query_info.prewhere_info) { diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index fb6d88c8d33..d19f8b7b16d 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -238,20 +238,9 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer if (!storage) { - if (query_info.force_tree_shaped_pipeline) - { - /// This flag means that pipeline must be tree-shaped, - /// so we can't enable processors for InterpreterSelectQuery here. - auto stream = InterpreterSelectQuery(modified_query_info.query, *modified_context, std::make_shared(header), - SelectQueryOptions(processed_stage).analyze()).execute().in; - - pipes.emplace_back(std::make_shared(std::move(stream))); - return pipes; - } - auto pipe = InterpreterSelectQuery(modified_query_info.query, *modified_context, std::make_shared(header), - SelectQueryOptions(processed_stage).analyze()).executeWithProcessors().getPipe(); + SelectQueryOptions(processed_stage).analyze()).execute().pipeline.getPipe(); pipe.addInterpreterContext(modified_context); pipes.emplace_back(std::move(pipe)); return pipes; @@ -276,15 +265,8 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer InterpreterSelectQuery interpreter{modified_query_info.query, *modified_context, SelectQueryOptions(processed_stage)}; - if (query_info.force_tree_shaped_pipeline) { - BlockInputStreamPtr stream = interpreter.execute().in; - Pipe pipe(std::make_shared(std::move(stream))); - pipes.emplace_back(std::move(pipe)); - } - else - { - Pipe pipe = interpreter.executeWithProcessors().getPipe(); + Pipe pipe = interpreter.execute().pipeline.getPipe(); pipes.emplace_back(std::move(pipe)); } From 2a9bcfbcd1f3acba8004ec5fe522dc695e4a0bc9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 14:00:12 +0300 Subject: [PATCH 036/150] Fix build. --- src/Interpreters/ClusterProxy/SelectStreamFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 4b35a982049..a97719427b0 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -115,7 +115,7 @@ void SelectStreamFactory::createForShard( const Cluster::ShardInfo & shard_info, const String &, const ASTPtr & query_ast, const Context & context, const ThrottlerPtr & throttler, - const SelectQueryInfo & query_info, + const SelectQueryInfo &, Pipes & res) { bool force_add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; From 557e531ff3ac333ef04444346c86b39d4e6ab7ab Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 15:06:31 +0300 Subject: [PATCH 037/150] Fix tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 3af128ffb00..eeee2df200c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -996,7 +996,7 @@ void InterpreterSelectQuery::executeFetchColumns( argument_types[j] = header.getByName(desc->argument_names[j]).type; Block block_with_count{ - {nullptr, std::make_shared(func, argument_types, desc->parameters), desc->column_name}}; + {ColumnPtr(column->cloneEmpty()), std::make_shared(func, argument_types, desc->parameters), desc->column_name}}; Chunk chunk(Columns(), column->size()); chunk.addColumn(std::move(column)); From 6b694d54b42b50fc2f07604a6a4be7b01b34a5f3 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 28 May 2020 15:39:55 +0200 Subject: [PATCH 038/150] Better kafka client.id --- src/Storages/Kafka/StorageKafka.cpp | 33 ++++++++++----- src/Storages/Kafka/StorageKafka.h | 3 +- tests/integration/test_storage_kafka/test.py | 43 +++++++++++++++++++- 3 files changed, 66 insertions(+), 13 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index a94aaca841b..143736b836d 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -34,6 +34,7 @@ #include #include #include +#include namespace DB @@ -133,7 +134,7 @@ StorageKafka::StorageKafka( , topics(global_context.getMacros()->expand(topics_)) , brokers(global_context.getMacros()->expand(brokers_)) , group(global_context.getMacros()->expand(group_)) - , client_id(global_context.getMacros()->expand(client_id_)) + , client_id(client_id_.empty() ? getDefaultClientId(table_id_) : global_context.getMacros()->expand(client_id_)) , format_name(global_context.getMacros()->expand(format_name_)) , row_delimiter(row_delimiter_) , schema_name(global_context.getMacros()->expand(schema_name_)) @@ -151,6 +152,13 @@ StorageKafka::StorageKafka( task->deactivate(); } +const String StorageKafka::getDefaultClientId(const StorageID & table_id_) const +{ + std::stringstream ss; + ss << VERSION_NAME << "-" << getFQDNOrHostName() << "-" << table_id_.database_name << "-" << table_id_.table_name; + return ss.str(); +} + Pipes StorageKafka::read( const Names & column_names, @@ -196,7 +204,7 @@ void StorageKafka::startup() { try { - pushReadBuffer(createReadBuffer()); + pushReadBuffer(createReadBuffer(i)); ++num_created_consumers; } catch (const cppkafka::Exception &) @@ -277,13 +285,22 @@ ProducerBufferPtr StorageKafka::createWriteBuffer(const Block & header) } -ConsumerBufferPtr StorageKafka::createReadBuffer() +ConsumerBufferPtr StorageKafka::createReadBuffer(const size_t consumer_number) { cppkafka::Configuration conf; conf.set("metadata.broker.list", brokers); conf.set("group.id", group); - conf.set("client.id", client_id); + if (num_consumers > 1) + { + std::stringstream ss; + ss << client_id << "-" << consumer_number; + conf.set("client.id", ss.str() ); + } + else + { + conf.set("client.id", client_id ); + } conf.set("auto.offset.reset", "smallest"); // If no offset stored for this group, read all messages from the start @@ -548,7 +565,6 @@ void registerStorageKafka(StorageFactory & factory) CHECK_KAFKA_STORAGE_ARGUMENT(8, kafka_max_block_size) CHECK_KAFKA_STORAGE_ARGUMENT(9, kafka_skip_broken_messages) CHECK_KAFKA_STORAGE_ARGUMENT(10, kafka_commit_every_batch) - CHECK_KAFKA_STORAGE_ARGUMENT(11, kafka_client_id) #undef CHECK_KAFKA_STORAGE_ARGUMENT @@ -714,12 +730,7 @@ void registerStorageKafka(StorageFactory & factory) } // Get and check client id - String client_id = kafka_settings.kafka_client_id.value; - if (args_count >= 11) - { - engine_args[10] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[10], args.local_context); - client_id = engine_args[10]->as().value.safeGet(); - } + String client_id = kafka_settings.kafka_client_id.value; return StorageKafka::create( args.table_id, args.context, args.columns, diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 0831218a534..977fe089792 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -110,12 +110,13 @@ private: BackgroundSchedulePool::TaskHolder task; std::atomic stream_cancelled{false}; - ConsumerBufferPtr createReadBuffer(); + ConsumerBufferPtr createReadBuffer(const size_t consumer_number); // Update Kafka configuration with values from CH user configuration. void updateConfiguration(cppkafka::Configuration & conf); void threadFunc(); + const String getDefaultClientId(const StorageID & table_id_) const; bool streamToViews(); bool checkDependencies(const StorageID & table_id); }; diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 13577864870..57551789b91 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -12,8 +12,11 @@ from helpers.network import PartitionManager import json import subprocess import kafka.errors -from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer +from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection from kafka.admin import NewTopic +from kafka.protocol.admin import DescribeGroupsResponse_v1, DescribeGroupsRequest_v1 +from kafka.protocol.group import MemberAssignment +import socket from google.protobuf.internal.encoder import _VarintBytes """ @@ -110,6 +113,32 @@ def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference else: return TSV(result) == TSV(reference) +# https://stackoverflow.com/a/57692111/1555175 +def describe_consumer_group(name): + client = BrokerConnection('localhost', 9092, socket.AF_INET) + client.connect_blocking() + + list_members_in_groups = DescribeGroupsRequest_v1(groups=[name]) + future = client.send(list_members_in_groups) + while not future.is_done: + for resp, f in client.recv(): + f.success(resp) + + (error_code, group_id, state, protocol_type, protocol, members) = future.value.groups[0] + + res = [] + for member in members: + (member_id, client_id, client_host, member_metadata, member_assignment) = member + member_info = {} + member_info['member_id'] = member_id + member_info['client_id'] = client_id + member_info['client_host'] = client_host + member_topics_assignment = [] + for (topic, partitions) in MemberAssignment.decode(member_assignment).assignment: + member_topics_assignment.append({'topic':topic, 'partitions':partitions}) + member_info['assignment'] = member_topics_assignment + res.append(member_info) + return res # Fixtures @@ -161,6 +190,9 @@ def test_kafka_settings_old_syntax(kafka_cluster): kafka_check_result(result, True) + members = describe_consumer_group('old') + assert members[0]['client_id'] == u'ClickHouse-instance-test-kafka' + # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) @pytest.mark.timeout(180) def test_kafka_settings_new_syntax(kafka_cluster): @@ -172,6 +204,7 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_group_name = 'new', kafka_format = 'JSONEachRow', kafka_row_delimiter = '\\n', + kafka_client_id = '{instance} test 1234', kafka_skip_broken_messages = 1; ''') @@ -197,6 +230,8 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_check_result(result, True) + members = describe_consumer_group('new') + assert members[0]['client_id'] == u'instance test 1234' @pytest.mark.timeout(180) def test_kafka_consumer_hang(kafka_cluster): @@ -837,6 +872,7 @@ def test_kafka_virtual_columns2(kafka_cluster): SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'virt2_0,virt2_1', kafka_group_name = 'virt2', + kafka_num_consumers = 2, kafka_format = 'JSONEachRow'; CREATE MATERIALIZED VIEW test.view Engine=Log AS @@ -866,6 +902,11 @@ def test_kafka_virtual_columns2(kafka_cluster): time.sleep(10) + members = describe_consumer_group('virt2') + #pprint.pprint(members) + members[0]['client_id'] = u'ClickHouse-instance-test-kafka-0' + members[1]['client_id'] = u'ClickHouse-instance-test-kafka-1' + result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) expected = '''\ From 4de1d6d421618963b330181c2f22f924f5daccb2 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 28 May 2020 15:53:27 +0200 Subject: [PATCH 039/150] Fixing style --- src/Storages/Kafka/StorageKafka.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 143736b836d..ec0b7b91319 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -295,11 +295,11 @@ ConsumerBufferPtr StorageKafka::createReadBuffer(const size_t consumer_number) { std::stringstream ss; ss << client_id << "-" << consumer_number; - conf.set("client.id", ss.str() ); + conf.set("client.id", ss.str()); } else { - conf.set("client.id", client_id ); + conf.set("client.id", client_id); } conf.set("auto.offset.reset", "smallest"); // If no offset stored for this group, read all messages from the start From 030aa6eac4301d9671afac9bfbaa5de61080946c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 16:57:01 +0300 Subject: [PATCH 040/150] Fix build. --- src/Interpreters/ClusterProxy/SelectStreamFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index a97719427b0..f8269c789ba 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -246,7 +246,7 @@ void SelectStreamFactory::createForShard( auto lazily_create_stream = [ pool = shard_info.pool, shard_num = shard_info.shard_num, modified_query, header = header, modified_query_ast, context, throttler, main_table = main_table, table_func_ptr = table_func_ptr, scalars = scalars, external_tables = external_tables, - stage = processed_stage, local_delay, add_totals_port, add_extremes_port]() + stage = processed_stage, local_delay]() -> BlockInputStreamPtr { auto current_settings = context.getSettingsRef(); From 28cc6d9c6b7628a86c4ca0eee46ed41f364ddcb9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 17:03:32 +0300 Subject: [PATCH 041/150] Try fix tests. --- src/Interpreters/MutationsInterpreter.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 6962885534c..224c3cab02b 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -184,7 +184,7 @@ bool isStorageTouchedByMutations( /// For some reason it may copy context and and give it into ExpressionBlockInputStream /// after that we will use context from destroyed stack frame in our stream. InterpreterSelectQuery interpreter(select_query, context_copy, storage, SelectQueryOptions().ignoreLimits()); - BlockInputStreamPtr in = interpreter.execute().in; + BlockInputStreamPtr in = interpreter.execute().getInputStream(); Block block = in->read(); if (!block.rows()) @@ -687,7 +687,7 @@ void MutationsInterpreter::validate(TableStructureReadLockHolder &) } /// Do not use getSampleBlock in order to check the whole pipeline. - Block first_stage_header = select_interpreter->execute().in->getHeader(); + Block first_stage_header = select_interpreter->execute().getInputStream()->getHeader(); BlockInputStreamPtr in = std::make_shared(first_stage_header); addStreamsForLaterStages(stages, in)->getHeader(); } @@ -697,7 +697,7 @@ BlockInputStreamPtr MutationsInterpreter::execute(TableStructureReadLockHolder & if (!can_execute) throw Exception("Cannot execute mutations interpreter because can_execute flag set to false", ErrorCodes::LOGICAL_ERROR); - BlockInputStreamPtr in = select_interpreter->execute().in; + BlockInputStreamPtr in = select_interpreter->execute().getInputStream(); auto result_stream = addStreamsForLaterStages(stages, in); From ea3f638b30a3c6199a9f11e5a53dfcca327c8253 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 17:11:36 +0300 Subject: [PATCH 042/150] Try fix tests. --- src/DataStreams/PushingToViewsBlockOutputStream.cpp | 2 +- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp | 6 +++--- src/Interpreters/ExpressionAnalyzer.cpp | 10 +++++----- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Interpreters/SubqueryForSet.cpp | 2 +- src/Storages/LiveView/StorageLiveView.cpp | 6 +++--- 7 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 7f730b5fd3f..414be847cdd 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -274,7 +274,7 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n StorageValues::create( storage->getStorageID(), storage->getColumns(), block, storage->getVirtuals())); select.emplace(view.query, local_context, SelectQueryOptions()); - in = std::make_shared(select->execute().in); + in = std::make_shared(select->execute().getInputStream()); /// Squashing is needed here because the materialized view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 7c2133e629f..cd5934b32b2 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -706,7 +706,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su { auto interpreter = interpretSubquery(right_in_operand, data.context, data.subquery_depth, {}); subquery_for_set.source = std::make_shared( - interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; }); + interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().getInputStream(); }); /** Why is LazyBlockInputStream used? * diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 0e9be3c7a96..e2884d99516 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -111,11 +111,11 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr } else { - BlockIO res = interpreter.execute(); + auto stream = interpreter.execute().getInputStream(); try { - block = res.in->read(); + block = stream->read(); if (!block) { @@ -126,7 +126,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr return; } - if (block.rows() != 1 || res.in->read()) + if (block.rows() != 1 || stream->read()) throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); } catch (const Exception & e) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b38af6feef9..27d6daf7292 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -297,13 +297,13 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr } auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, {}, query_options); - BlockIO res = interpreter_subquery->execute(); + auto stream = interpreter_subquery->execute().getInputStream(); SetPtr set = std::make_shared(settings.size_limits_for_set, true, context.getSettingsRef().transform_null_in); - set->setHeader(res.in->getHeader()); + set->setHeader(stream->getHeader()); - res.in->readPrefix(); - while (Block block = res.in->read()) + stream->readPrefix(); + while (Block block = stream->read()) { /// If the limits have been exceeded, give up and let the default subquery processing actions take place. if (!set->insertFromBlock(block)) @@ -311,7 +311,7 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr } set->finishInsert(); - res.in->readSuffix(); + stream->readSuffix(); prepared_sets[set_key] = std::move(set); } diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 37a358c3d28..65dd3cf2667 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -134,7 +134,7 @@ public: ast = database_and_table_name; external_tables[external_table_name] = external_storage_holder; - subqueries_for_sets[external_table_name].source = interpreter->execute().in; + subqueries_for_sets[external_table_name].source = interpreter->execute().getInputStream(); subqueries_for_sets[external_table_name].table = external_storage; /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/SubqueryForSet.cpp b/src/Interpreters/SubqueryForSet.cpp index 72831970de7..ac5c1e3d9eb 100644 --- a/src/Interpreters/SubqueryForSet.cpp +++ b/src/Interpreters/SubqueryForSet.cpp @@ -13,7 +13,7 @@ void SubqueryForSet::makeSource(std::shared_ptr { joined_block_aliases = std::move(joined_block_aliases_); source = std::make_shared(interpreter->getSampleBlock(), - [interpreter]() mutable { return interpreter->execute().in; }); + [interpreter]() mutable { return interpreter->execute().getInputStream(); }); sample_block = source->getHeader(); renameColumns(sample_block); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index cd660407c89..c1372e2c8c4 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -111,7 +111,7 @@ MergeableBlocksPtr StorageLiveView::collectMergeableBlocks(const Context & conte InterpreterSelectQuery interpreter(mergeable_query->clone(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); - auto view_mergeable_stream = std::make_shared(interpreter.execute().in); + auto view_mergeable_stream = std::make_shared(interpreter.execute().getInputStream()); while (Block this_block = view_mergeable_stream->read()) base_blocks->push_back(this_block); @@ -148,7 +148,7 @@ BlockInputStreamPtr StorageLiveView::completeQuery(Pipes pipes) block_context->addExternalTable(getBlocksTableName(), TemporaryTableHolder(global_context, creator)); InterpreterSelectQuery select(getInnerBlocksQuery(), *block_context, StoragePtr(), SelectQueryOptions(QueryProcessingStage::Complete)); - BlockInputStreamPtr data = std::make_shared(select.execute().in); + BlockInputStreamPtr data = std::make_shared(select.execute().getInputStream()); /// Squashing is needed here because the view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY @@ -218,7 +218,7 @@ void StorageLiveView::writeIntoLiveView( QueryProcessingStage::WithMergeableState); auto data_mergeable_stream = std::make_shared( - select_block.execute().in); + select_block.execute().getInputStream()); while (Block this_block = data_mergeable_stream->read()) new_mergeable_blocks->push_back(this_block); From 34a93197f7d2342508eaafdcde1b39836517b47a Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 28 May 2020 16:52:03 +0200 Subject: [PATCH 043/150] Update librdkafka to version 1.4.2 --- contrib/librdkafka | 2 +- contrib/librdkafka-cmake/CMakeLists.txt | 14 ++++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/contrib/librdkafka b/contrib/librdkafka index 4ffe54b4f59..b0d91bd74ab 160000 --- a/contrib/librdkafka +++ b/contrib/librdkafka @@ -1 +1 @@ -Subproject commit 4ffe54b4f59ee5ae3767f9f25dc14651a3384d62 +Subproject commit b0d91bd74abb5f0e1ee972d326a317ad610f6300 diff --git a/contrib/librdkafka-cmake/CMakeLists.txt b/contrib/librdkafka-cmake/CMakeLists.txt index 0f7b8e349ff..93ef9d2357b 100644 --- a/contrib/librdkafka-cmake/CMakeLists.txt +++ b/contrib/librdkafka-cmake/CMakeLists.txt @@ -3,20 +3,30 @@ set(RDKAFKA_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/librdkafka/src) set(SRCS ${RDKAFKA_SOURCE_DIR}/crc32c.c ${RDKAFKA_SOURCE_DIR}/rdkafka_zstd.c +# ${RDKAFKA_SOURCE_DIR}/lz4.c +# ${RDKAFKA_SOURCE_DIR}/lz4frame.c +# ${RDKAFKA_SOURCE_DIR}/lz4hc.c +# ${RDKAFKA_SOURCE_DIR}/rdxxhash.c +# ${RDKAFKA_SOURCE_DIR}/regexp.c ${RDKAFKA_SOURCE_DIR}/rdaddr.c ${RDKAFKA_SOURCE_DIR}/rdavl.c ${RDKAFKA_SOURCE_DIR}/rdbuf.c ${RDKAFKA_SOURCE_DIR}/rdcrc32.c ${RDKAFKA_SOURCE_DIR}/rddl.c + ${RDKAFKA_SOURCE_DIR}/rdfnv1a.c ${RDKAFKA_SOURCE_DIR}/rdhdrhistogram.c ${RDKAFKA_SOURCE_DIR}/rdkafka.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_admin.c # looks optional ${RDKAFKA_SOURCE_DIR}/rdkafka_assignor.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_aux.c # looks optional ${RDKAFKA_SOURCE_DIR}/rdkafka_background.c ${RDKAFKA_SOURCE_DIR}/rdkafka_broker.c ${RDKAFKA_SOURCE_DIR}/rdkafka_buf.c ${RDKAFKA_SOURCE_DIR}/rdkafka_cert.c ${RDKAFKA_SOURCE_DIR}/rdkafka_cgrp.c ${RDKAFKA_SOURCE_DIR}/rdkafka_conf.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_coord.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_error.c ${RDKAFKA_SOURCE_DIR}/rdkafka_event.c ${RDKAFKA_SOURCE_DIR}/rdkafka_feature.c ${RDKAFKA_SOURCE_DIR}/rdkafka_idempotence.c @@ -24,6 +34,7 @@ set(SRCS ${RDKAFKA_SOURCE_DIR}/rdkafka_metadata.c ${RDKAFKA_SOURCE_DIR}/rdkafka_metadata_cache.c ${RDKAFKA_SOURCE_DIR}/rdkafka_mock.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_mock_cgrp.c ${RDKAFKA_SOURCE_DIR}/rdkafka_mock_handlers.c ${RDKAFKA_SOURCE_DIR}/rdkafka_msg.c ${RDKAFKA_SOURCE_DIR}/rdkafka_msgset_reader.c @@ -38,9 +49,11 @@ set(SRCS ${RDKAFKA_SOURCE_DIR}/rdkafka_request.c ${RDKAFKA_SOURCE_DIR}/rdkafka_roundrobin_assignor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl.c +# ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_cyrus.c # needed to support Kerberos, requires cyrus-sasl ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_oauthbearer.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_plain.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_scram.c +# ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_win32.c ${RDKAFKA_SOURCE_DIR}/rdkafka_ssl.c ${RDKAFKA_SOURCE_DIR}/rdkafka_subscription.c ${RDKAFKA_SOURCE_DIR}/rdkafka_timer.c @@ -48,6 +61,7 @@ set(SRCS ${RDKAFKA_SOURCE_DIR}/rdkafka_transport.c ${RDKAFKA_SOURCE_DIR}/rdkafka_interceptor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_header.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_txnmgr.c ${RDKAFKA_SOURCE_DIR}/rdlist.c ${RDKAFKA_SOURCE_DIR}/rdlog.c ${RDKAFKA_SOURCE_DIR}/rdmurmur2.c From c6f82590ad8a0a4cd56fd7458748d30af1a18fea Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 28 May 2020 18:34:33 +0300 Subject: [PATCH 044/150] Add missed files --- src/Storages/TTLDescription.cpp | 197 ++++++++++++++++++++++++++++++++ src/Storages/TTLDescription.h | 69 +++++++++++ 2 files changed, 266 insertions(+) create mode 100644 src/Storages/TTLDescription.cpp create mode 100644 src/Storages/TTLDescription.h diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp new file mode 100644 index 00000000000..0efe6076174 --- /dev/null +++ b/src/Storages/TTLDescription.cpp @@ -0,0 +1,197 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include + + +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int BAD_TTL_EXPRESSION; +} + +namespace +{ + +void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const String & result_column_name) +{ + for (const auto & action : ttl_expression->getActions()) + { + if (action.type == ExpressionAction::APPLY_FUNCTION) + { + IFunctionBase & func = *action.function_base; + if (!func.isDeterministic()) + throw Exception( + "TTL expression cannot contain non-deterministic functions, " + "but contains function " + + func.getName(), + ErrorCodes::BAD_ARGUMENTS); + } + } + + const auto & result_column = ttl_expression->getSampleBlock().getByName(result_column_name); + + if (!typeid_cast(result_column.type.get()) + && !typeid_cast(result_column.type.get())) + { + throw Exception( + "TTL expression result column should have DateTime or Date type, but has " + result_column.type->getName(), + ErrorCodes::BAD_TTL_EXPRESSION); + } +} + +} + +TTLDescription TTLDescription::getTTLFromAST( + const ASTPtr & definition_ast, + const ColumnsDescription & columns, + const Context & context, + const StorageMetadataKeyField & primary_key) +{ + TTLDescription result; + const auto * ttl_element = definition_ast->as(); + + /// First child is expression: `TTL expr TO DISK` + if (ttl_element != nullptr) + result.expression_ast = ttl_element->children.front()->clone(); + else /// It's columns TTL without any additions, just copy it + result.expression_ast = definition_ast->clone(); + + auto ttl_ast = result.expression_ast->clone(); + auto syntax_analyzer_result = SyntaxAnalyzer(context).analyze(ttl_ast, columns.getAllPhysical()); + result.expression = ExpressionAnalyzer(ttl_ast, syntax_analyzer_result, context).getActions(false); + result.result_column = ttl_ast->getColumnName(); + + if (ttl_element == nullptr) /// columns TTL + { + result.destination_type = DataDestinationType::DELETE; + result.mode = TTLMode::DELETE; + } + else /// rows TTL + { + result.destination_type = ttl_element->destination_type; + result.destination_name = ttl_element->destination_name; + result.mode = ttl_element->mode; + + if (ttl_element->mode == TTLMode::DELETE) + { + if (ASTPtr where_expr_ast = ttl_element->where()) + { + auto where_syntax_result = SyntaxAnalyzer(context).analyze(where_expr_ast, columns.getAllPhysical()); + result.where_expression = ExpressionAnalyzer(where_expr_ast, where_syntax_result, context).getActions(false); + result.where_result_column = where_expr_ast->getColumnName(); + } + } + else if (ttl_element->mode == TTLMode::GROUP_BY) + { + const auto & pk_columns = primary_key.column_names; + + if (ttl_element->group_by_key.size() > pk_columns.size()) + throw Exception("TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); + + NameSet primary_key_columns_set(pk_columns.begin(), pk_columns.end()); + NameSet aggregation_columns_set; + + for (const auto & column : primary_key.expression->getRequiredColumns()) + primary_key_columns_set.insert(column); + + for (size_t i = 0; i < ttl_element->group_by_key.size(); ++i) + { + if (ttl_element->group_by_key[i]->getColumnName() != pk_columns[i]) + throw Exception( + "TTL Expression GROUP BY key should be a prefix of primary key", + ErrorCodes::BAD_TTL_EXPRESSION); + } + + for (const auto & [name, value] : ttl_element->group_by_aggregations) + { + if (primary_key_columns_set.count(name)) + throw Exception( + "Can not set custom aggregation for column in primary key in TTL Expression", + ErrorCodes::BAD_TTL_EXPRESSION); + + aggregation_columns_set.insert(name); + } + + if (aggregation_columns_set.size() != ttl_element->group_by_aggregations.size()) + throw Exception( + "Multiple aggregations set for one column in TTL Expression", + ErrorCodes::BAD_TTL_EXPRESSION); + + + result.group_by_keys = Names(pk_columns.begin(), pk_columns.begin() + ttl_element->group_by_key.size()); + + auto aggregations = ttl_element->group_by_aggregations; + + for (size_t i = 0; i < pk_columns.size(); ++i) + { + ASTPtr value = primary_key.expression_list_ast->children[i]->clone(); + + if (i >= ttl_element->group_by_key.size()) + { + ASTPtr value_max = makeASTFunction("max", value->clone()); + aggregations.emplace_back(value->getColumnName(), std::move(value_max)); + } + + if (value->as()) + { + auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, true); + auto expr_actions = ExpressionAnalyzer(value, syntax_result, context).getActions(false); + for (const auto & column : expr_actions->getRequiredColumns()) + { + if (i < ttl_element->group_by_key.size()) + { + ASTPtr expr = makeASTFunction("any", std::make_shared(column)); + aggregations.emplace_back(column, std::move(expr)); + } + else + { + ASTPtr expr = makeASTFunction("argMax", std::make_shared(column), value->clone()); + aggregations.emplace_back(column, std::move(expr)); + } + } + } + } + + for (const auto & column : columns.getAllPhysical()) + { + if (!primary_key_columns_set.count(column.name) && !aggregation_columns_set.count(column.name)) + { + ASTPtr expr = makeASTFunction("any", std::make_shared(column.name)); + aggregations.emplace_back(column.name, std::move(expr)); + } + } + + for (auto [name, value] : aggregations) + { + auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, true); + auto expr_analyzer = ExpressionAnalyzer(value, syntax_result, context); + + result.group_by_aggregations.emplace_back(name, value->getColumnName(), expr_analyzer.getActions(false)); + + for (const auto & descr : expr_analyzer.getAnalyzedData().aggregate_descriptions) + result.aggregate_descriptions.push_back(descr); + } + } + } + + checkTTLExpression(result.expression, result.result_column); + + + return result; +} + +} diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h new file mode 100644 index 00000000000..59829dbecfd --- /dev/null +++ b/src/Storages/TTLDescription.h @@ -0,0 +1,69 @@ +#pragma once +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/// Common struct for TTL record in storage +struct TTLDescription +{ + TTLMode mode; + + /// Expression part of TTL AST: + /// TTL d + INTERVAL 1 DAY + /// ^~~~~expression~~~~^ + ASTPtr expression_ast; + + /// Expresion actions evaluated from AST + ExpressionActionsPtr expression; + + /// Result column of this TTL expression + String result_column; + + ExpressionActionsPtr where_expression; + + String where_result_column; + + Names group_by_keys; + + std::vector> group_by_aggregations; + AggregateDescriptions aggregate_descriptions; + + /// Destination type, only valid for table TTLs. + /// For example DISK or VOLUME + DataDestinationType destination_type; + + /// Name of destination disk or volume + String destination_name; + + /// Parse TTL structure from definition. Able to parse both column and table + /// TTLs. + static TTLDescription getTTLFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context, const StorageMetadataKeyField & primary_key); +}; + +/// Mapping from column name to column TTL +using TTLColumnsDescription = std::unordered_map; +using TTLDescriptions = std::vector; + +/// Common TTL for all table. Specified after defining the table columns. +struct TTLTableDescription +{ + /// Definition. Include all parts of TTL: + /// TTL d + INTERVAL 1 day TO VOLUME 'disk1' + /// ^~~~~~~~~~~~~~~definition~~~~~~~~~~~~~~~^ + ASTPtr definition_ast; + + /// Rows removing TTL + TTLDescription rows_ttl; + + /// Moving data TTL (to other disks or volumes) + TTLDescriptions move_ttl; +}; + +} From 534e55f5b54f19be03cc8338a832d91bdd968c02 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 19:01:12 +0300 Subject: [PATCH 045/150] try fix tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index eeee2df200c..9b161bc1883 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -996,13 +996,10 @@ void InterpreterSelectQuery::executeFetchColumns( argument_types[j] = header.getByName(desc->argument_names[j]).type; Block block_with_count{ - {ColumnPtr(column->cloneEmpty()), std::make_shared(func, argument_types, desc->parameters), desc->column_name}}; + {std::move(column), std::make_shared(func, argument_types, desc->parameters), desc->column_name}}; - Chunk chunk(Columns(), column->size()); - chunk.addColumn(std::move(column)); - - auto source = std::make_shared(std::move(block_with_count), std::move(chunk)); - pipeline.init(Pipe(std::move(source))); + auto istream = std::make_shared(block_with_count); + pipeline.init(Pipe(std::make_shared(istream))); from_stage = QueryProcessingStage::WithMergeableState; analysis_result.first_stage = false; return; From 51be859b16771406be557b477125d8e310d8546b Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Thu, 28 May 2020 19:11:27 +0300 Subject: [PATCH 046/150] S3 Poco HTTP Client WIP (timeouts + proxy support). --- src/IO/HTTPCommon.cpp | 11 ++- src/IO/HTTPCommon.h | 2 +- src/IO/S3/PocoHttpClient.cpp | 109 +++++++++++++++------------- src/IO/S3/PocoHttpClient.h | 4 + src/IO/S3/PocoHttpClientFactory.cpp | 4 +- 5 files changed, 72 insertions(+), 58 deletions(-) diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 14c97ee56f4..47f058c484b 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -68,7 +68,7 @@ namespace throw Exception("Unsupported scheme in URI '" + uri.toString() + "'", ErrorCodes::UNSUPPORTED_URI_SCHEME); } - HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive) + HTTPSessionPtr makeHTTPSessionImpl(const std::string & host, UInt16 port, bool https, bool keep_alive, bool resolve_host=true) { HTTPSessionPtr session; @@ -83,7 +83,10 @@ namespace ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections); - session->setHost(DNSResolver::instance().resolveHost(host).toString()); + if (resolve_host) + session->setHost(DNSResolver::instance().resolveHost(host).toString()); + else + session->setHost(host); session->setPort(port); /// doesn't work properly without patch @@ -202,13 +205,13 @@ void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, unsigne response.set("Keep-Alive", "timeout=" + std::to_string(timeout.totalSeconds())); } -HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts) +HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, bool resolve_host) { const std::string & host = uri.getHost(); UInt16 port = uri.getPort(); bool https = isHTTPS(uri); - auto session = makeHTTPSessionImpl(host, port, https, false); + auto session = makeHTTPSessionImpl(host, port, https, false, resolve_host); setTimeouts(*session, timeouts); return session; } diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index 7592c1c31b3..66764b1c805 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -45,7 +45,7 @@ using HTTPSessionPtr = std::shared_ptr; void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, unsigned keep_alive_timeout); /// Create session object to perform requests and set required parameters. -HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts); +HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, bool resolve_host = true); /// As previous method creates session, but tooks it from pool PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size); diff --git a/src/IO/S3/PocoHttpClient.cpp b/src/IO/S3/PocoHttpClient.cpp index 9e2ef40fe50..f7b71c956a7 100644 --- a/src/IO/S3/PocoHttpClient.cpp +++ b/src/IO/S3/PocoHttpClient.cpp @@ -1,12 +1,12 @@ #include "PocoHttpClient.h" +#include +#include #include #include #include -#include #include -#include -#include +#include #include "Poco/StreamCopier.h" #include #include @@ -14,6 +14,16 @@ namespace DB::S3 { +PocoHttpClient::PocoHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration) + : per_request_configuration(clientConfiguration.perRequestConfiguration) + , timeouts(ConnectionTimeouts( + Poco::Timespan(clientConfiguration.connectTimeoutMs * 1000), /// connection timeout. + Poco::Timespan(clientConfiguration.httpRequestTimeoutMs * 1000), /// send timeout. + Poco::Timespan(clientConfiguration.httpRequestTimeoutMs * 1000) /// receive timeout. + )) +{ +} + std::shared_ptr PocoHttpClient::MakeRequest( Aws::Http::HttpRequest & request, Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, @@ -41,7 +51,6 @@ void PocoHttpClient::MakeRequestInternal( Aws::Utils::RateLimits::RateLimiterInterface *) const { auto uri = request.GetUri().GetURIString(); - LOG_DEBUG(&Logger::get("AWSClient"), "Make request to: {}", uri); const int MAX_REDIRECT_ATTEMPTS = 10; @@ -49,73 +58,69 @@ void PocoHttpClient::MakeRequestInternal( { for (int attempt = 0; attempt < MAX_REDIRECT_ATTEMPTS; ++attempt) { - /// 1 second is enough for now. - /// TODO: Make timeouts configurable. - ConnectionTimeouts timeouts( - Poco::Timespan(2000000), /// Connection timeout. - Poco::Timespan(2000000), /// Send timeout. - Poco::Timespan(2000000) /// Receive timeout. - ); - auto session = makeHTTPSession(Poco::URI(uri), timeouts); + Poco::URI poco_uri(uri); - Poco::Net::HTTPRequest request_(Poco::Net::HTTPRequest::HTTP_1_1); + /// Reverse proxy can replace host header with resolved ip address instead of host name. + /// This can lead to request signature difference on S3 side. + auto session = makeHTTPSession(poco_uri, timeouts, false); - request_.setURI(uri); + auto request_configuration = per_request_configuration(request); + if (!request_configuration.proxyHost.empty()) + session->setProxy(request_configuration.proxyHost, request_configuration.proxyPort); + + Poco::Net::HTTPRequest poco_request(Poco::Net::HTTPRequest::HTTP_1_1); + + poco_request.setURI(poco_uri.getPathAndQuery()); switch (request.GetMethod()) { case Aws::Http::HttpMethod::HTTP_GET: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_GET); + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_GET); break; case Aws::Http::HttpMethod::HTTP_POST: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_POST); + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_POST); break; case Aws::Http::HttpMethod::HTTP_DELETE: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_DELETE); + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_DELETE); break; case Aws::Http::HttpMethod::HTTP_PUT: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_PUT); + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_PUT); break; case Aws::Http::HttpMethod::HTTP_HEAD: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_HEAD); + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_HEAD); break; case Aws::Http::HttpMethod::HTTP_PATCH: - request_.setMethod(Poco::Net::HTTPRequest::HTTP_PATCH); + poco_request.setMethod(Poco::Net::HTTPRequest::HTTP_PATCH); break; } for (const auto & [header_name, header_value] : request.GetHeaders()) - request_.set(header_name, header_value); + poco_request.set(header_name, header_value); - request_.setExpectContinue(true); + Poco::Net::HTTPResponse poco_response; + auto & request_body_stream = session->sendRequest(poco_request); - Poco::Net::HTTPResponse response_; - auto & request_body_stream = session->sendRequest(request_); - - if (session->peekResponse(response_)) + if (request.GetContentBody()) { - if (request.GetContentBody()) + LOG_DEBUG(&Logger::get("AWSClient"), "Writing request body..."); + if (attempt > 0) /// rewind content body buffer. { - if (attempt > 0) /// rewind content body buffer. - { - request.GetContentBody()->clear(); - request.GetContentBody()->seekg(0); - } - auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); - LOG_DEBUG( - &Logger::get("AWSClient"), "Written {} bytes to request body", size); + request.GetContentBody()->clear(); + request.GetContentBody()->seekg(0); } + auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); + LOG_DEBUG(&Logger::get("AWSClient"), "Written {} bytes to request body", size); } - auto & response_body_stream = session->receiveResponse(response_); + LOG_DEBUG(&Logger::get("AWSClient"), "Receiving response..."); + auto & response_body_stream = session->receiveResponse(poco_response); - int status_code = static_cast(response_.getStatus()); - LOG_DEBUG( - &Logger::get("AWSClient"), "Response status: {}, {}", status_code, response_.getReason()); + int status_code = static_cast(poco_response.getStatus()); + LOG_DEBUG(&Logger::get("AWSClient"), "Response status: {}, {}", status_code, poco_response.getReason()); - if (response_.getStatus() == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) + if (poco_response.getStatus() == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) { - auto location = response_.get("location"); + auto location = poco_response.get("location"); uri = location; LOG_DEBUG(&Logger::get("AWSClient"), "Redirecting request to new location: {}", location); @@ -123,25 +128,27 @@ void PocoHttpClient::MakeRequestInternal( } response->SetResponseCode(static_cast(status_code)); - response->SetContentType(response_.getContentType()); + response->SetContentType(poco_response.getContentType()); std::stringstream headers_ss; - for (const auto & [header_name, header_value] : response_) + for (const auto & [header_name, header_value] : poco_response) { response->AddHeader(header_name, header_value); - headers_ss << " " << header_name << " : " << header_value << ";"; + headers_ss << header_name << " : " << header_value << "; "; } - - LOG_DEBUG(&Logger::get("AWSClient"), "Received headers:{}", headers_ss.str()); - - /// TODO: Do not copy whole stream. - Poco::StreamCopier::copyStream(response_body_stream, response->GetResponseBody()); + LOG_DEBUG(&Logger::get("AWSClient"), "Received headers: {}", headers_ss.str()); if (status_code >= 300) { + String error_message; + Poco::StreamCopier::copyToString(response_body_stream, error_message); + response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); - response->SetClientErrorMessage(response_.getReason()); + response->SetClientErrorMessage(error_message); } + else + /// TODO: Do not copy whole stream. + Poco::StreamCopier::copyStream(response_body_stream, response->GetResponseBody()); break; } @@ -150,7 +157,7 @@ void PocoHttpClient::MakeRequestInternal( { tryLogCurrentException(&Logger::get("AWSClient"), "Failed to make request to: " + uri); response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); - response->SetClientErrorMessage(getCurrentExceptionMessage(true)); + response->SetClientErrorMessage(getCurrentExceptionMessage(false)); } } } diff --git a/src/IO/S3/PocoHttpClient.h b/src/IO/S3/PocoHttpClient.h index 5de20c9d978..7458c692a0e 100644 --- a/src/IO/S3/PocoHttpClient.h +++ b/src/IO/S3/PocoHttpClient.h @@ -1,5 +1,6 @@ #pragma once +#include #include namespace Aws::Http::Standard @@ -13,6 +14,7 @@ namespace DB::S3 class PocoHttpClient : public Aws::Http::HttpClient { public: + explicit PocoHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration); ~PocoHttpClient() override = default; std::shared_ptr MakeRequest( Aws::Http::HttpRequest & request, @@ -31,6 +33,8 @@ private: Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const; + std::function per_request_configuration; + ConnectionTimeouts timeouts; }; } diff --git a/src/IO/S3/PocoHttpClientFactory.cpp b/src/IO/S3/PocoHttpClientFactory.cpp index e72f9a8a4a4..96a73e1d001 100644 --- a/src/IO/S3/PocoHttpClientFactory.cpp +++ b/src/IO/S3/PocoHttpClientFactory.cpp @@ -9,9 +9,9 @@ namespace DB::S3 { std::shared_ptr -PocoHttpClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration &) const +PocoHttpClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration) const { - return std::make_shared(); + return std::make_shared(clientConfiguration); } std::shared_ptr PocoHttpClientFactory::CreateHttpRequest( From acc3e6b15a82c1ca93eb7e2061c00ae5a5ca72a8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 28 May 2020 19:31:27 +0300 Subject: [PATCH 047/150] Better structure --- src/DataStreams/TTLBlockInputStream.cpp | 10 ++++---- src/Storages/TTLDescription.cpp | 3 ++- src/Storages/TTLDescription.h | 32 +++++++++++++++++++++++-- src/Storages/ya.make | 1 + 4 files changed, 38 insertions(+), 8 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 87cab76ce8c..c6542763533 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -277,18 +277,18 @@ void TTLBlockInputStream::finalizeAggregates(MutableColumns & result_columns) auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1); for (auto & agg_block : aggregated_res) { - for (const auto & it : storage.getRowsTTL().group_by_aggregations) - std::get<2>(it)->execute(agg_block); + for (const auto & it : storage.getRowsTTL().set_parts) + it.expression->execute(agg_block); for (const auto & name : storage.getRowsTTL().group_by_keys) { const IColumn * values_column = agg_block.getByName(name).column.get(); auto & result_column = result_columns[header.getPositionByName(name)]; result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); } - for (const auto & it : storage.getRowsTTL().group_by_aggregations) + for (const auto & it : storage.getRowsTTL().set_parts) { - const IColumn * values_column = agg_block.getByName(get<1>(it)).column.get(); - auto & result_column = result_columns[header.getPositionByName(std::get<0>(it))]; + const IColumn * values_column = agg_block.getByName(it.expression_result_column_name).column.get(); + auto & result_column = result_columns[header.getPositionByName(it.column_name)]; result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); } } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 0efe6076174..92fcf400dd3 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -180,7 +180,8 @@ TTLDescription TTLDescription::getTTLFromAST( auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, true); auto expr_analyzer = ExpressionAnalyzer(value, syntax_result, context); - result.group_by_aggregations.emplace_back(name, value->getColumnName(), expr_analyzer.getActions(false)); + result.set_parts.emplace_back(TTLSetPartDescription{ + name, value->getColumnName(), expr_analyzer.getActions(false)}); for (const auto & descr : expr_analyzer.getAnalyzedData().aggregate_descriptions) result.aggregate_descriptions.push_back(descr); diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 59829dbecfd..d0e669ef4cf 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -10,6 +10,24 @@ namespace DB { +struct TTLSetPartDescription +{ + /// Name of column in set part of ttl expression + /// x = sum(y) + /// ^ + String column_name; + + /// Name of column on the right hand of the set part of TTL expression + /// x = sum(y) + /// ^~~~~~^ + String expression_result_column_name; + + /// Expressions to calculate the value of set expression + ExpressionActionsPtr expression; +}; + +using TTLSetPartDescriptions = std::vector; + /// Common struct for TTL record in storage struct TTLDescription { @@ -17,7 +35,7 @@ struct TTLDescription /// Expression part of TTL AST: /// TTL d + INTERVAL 1 DAY - /// ^~~~~expression~~~~^ + /// ^~~~~~~~~~~~~~~~~~~^ ASTPtr expression_ast; /// Expresion actions evaluated from AST @@ -26,13 +44,23 @@ struct TTLDescription /// Result column of this TTL expression String result_column; + /// WHERE part in TTL expression + /// TTL ... WHERE x % 10 == 0 and y > 5 + /// ^~~~~~~~~~~~~~~~~~~~~~^ ExpressionActionsPtr where_expression; + /// Name of result column from WHERE expression String where_result_column; + /// Names of key columns in GROUP BY expression + /// TTL ... GROUP BY toDate(d), x SET ... + /// ^~~~~~~~~~~~^ Names group_by_keys; - std::vector> group_by_aggregations; + /// SET parts of TTL expression + TTLSetPartDescriptions set_parts; + + /// Aggregate descriptions for GROUP BY in TTL AggregateDescriptions aggregate_descriptions; /// Destination type, only valid for table TTLs. diff --git a/src/Storages/ya.make b/src/Storages/ya.make index ffa3924d11a..3e6943bac69 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -173,6 +173,7 @@ SRCS( StorageXDBC.cpp transformQueryForExternalDatabase.cpp VirtualColumnUtils.cpp + TTLDescriptions.cpp ) END() From c5fe87e123dca8a5fdcaace0f349209e51c9c11b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 28 May 2020 19:32:35 +0300 Subject: [PATCH 048/150] Less includes --- src/Storages/StorageInMemoryMetadata.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 0111c0632c8..6713519151f 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -5,22 +5,11 @@ #include #include #include -#include -#include - -#include -#include namespace DB { -namespace ErrorCodes -{ - extern const int BAD_TTL_EXPRESSION; - extern const int BAD_ARGUMENTS; -}; - StorageInMemoryMetadata::StorageInMemoryMetadata( const ColumnsDescription & columns_, const IndicesDescription & indices_, From eb5a8dccfa3c090bbb45ccff787aa9383e1341a9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 28 May 2020 20:32:22 +0300 Subject: [PATCH 049/150] Fix ya.make --- src/Storages/ya.make | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 3e6943bac69..8a36fad696f 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -173,7 +173,7 @@ SRCS( StorageXDBC.cpp transformQueryForExternalDatabase.cpp VirtualColumnUtils.cpp - TTLDescriptions.cpp + TTLDescription.cpp ) END() From e39c4cdf7121a4e766ddebd777ee9979b06ec004 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 28 May 2020 19:41:57 +0200 Subject: [PATCH 050/150] Add _timestamp_ms virtual columns to Kafka engine --- src/Storages/Kafka/KafkaBlockInputStream.cpp | 12 ++++--- src/Storages/Kafka/StorageKafka.cpp | 5 ++- tests/integration/test_storage_kafka/test.py | 34 ++++++++++---------- 3 files changed, 28 insertions(+), 23 deletions(-) diff --git a/src/Storages/Kafka/KafkaBlockInputStream.cpp b/src/Storages/Kafka/KafkaBlockInputStream.cpp index a2403e66c50..6ae7e2606b6 100644 --- a/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -19,8 +19,8 @@ KafkaBlockInputStream::KafkaBlockInputStream( , column_names(columns) , max_block_size(max_block_size_) , commit_in_suffix(commit_in_suffix_) - , non_virtual_header(storage.getSampleBlockNonMaterialized()) /// FIXME: add materialized columns support - , virtual_header(storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp"})) + , non_virtual_header(storage.getSampleBlockNonMaterialized()) + , virtual_header(storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp","_timestamp_ms"})) { context.setSetting("input_format_skip_unknown_fields", 1u); // Always skip unknown fields regardless of the context (JSON or TSKV) @@ -141,8 +141,7 @@ Block KafkaBlockInputStream::readImpl() auto offset = buffer->currentOffset(); auto partition = buffer->currentPartition(); auto timestamp_raw = buffer->currentTimestamp(); - auto timestamp = timestamp_raw ? std::chrono::duration_cast(timestamp_raw->get_timestamp()).count() - : 0; + for (size_t i = 0; i < new_rows; ++i) { virtual_columns[0]->insert(topic); @@ -151,11 +150,14 @@ Block KafkaBlockInputStream::readImpl() virtual_columns[3]->insert(partition); if (timestamp_raw) { - virtual_columns[4]->insert(timestamp); + auto ts = timestamp_raw->get_timestamp(); + virtual_columns[4]->insert(std::chrono::duration_cast(ts).count()); + virtual_columns[5]->insert(DecimalField(std::chrono::duration_cast(ts).count(),3)); } else { virtual_columns[4]->insertDefault(); + virtual_columns[5]->insertDefault(); } } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 7731cf3c06a..c2d26dfa300 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -36,6 +37,7 @@ #include + namespace DB { @@ -724,7 +726,8 @@ NamesAndTypesList StorageKafka::getVirtuals() const {"_key", std::make_shared()}, {"_offset", std::make_shared()}, {"_partition", std::make_shared()}, - {"_timestamp", std::make_shared(std::make_shared())} + {"_timestamp", std::make_shared(std::make_shared())}, + {"_timestamp_ms", std::make_shared(std::make_shared(3))} }; } diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 13577864870..d89684e2131 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -840,28 +840,28 @@ def test_kafka_virtual_columns2(kafka_cluster): kafka_format = 'JSONEachRow'; CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp) FROM test.kafka; + SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms) FROM test.kafka; ''') producer = KafkaProducer(bootstrap_servers="localhost:9092") - producer.send(topic='virt2_0', value=json.dumps({'value': 1}), partition=0, key='k1', timestamp_ms=1577836801000) - producer.send(topic='virt2_0', value=json.dumps({'value': 2}), partition=0, key='k2', timestamp_ms=1577836802000) + producer.send(topic='virt2_0', value=json.dumps({'value': 1}), partition=0, key='k1', timestamp_ms=1577836801001) + producer.send(topic='virt2_0', value=json.dumps({'value': 2}), partition=0, key='k2', timestamp_ms=1577836802002) producer.flush() time.sleep(1) - producer.send(topic='virt2_0', value=json.dumps({'value': 3}), partition=1, key='k3', timestamp_ms=1577836803000) - producer.send(topic='virt2_0', value=json.dumps({'value': 4}), partition=1, key='k4', timestamp_ms=1577836804000) + producer.send(topic='virt2_0', value=json.dumps({'value': 3}), partition=1, key='k3', timestamp_ms=1577836803003) + producer.send(topic='virt2_0', value=json.dumps({'value': 4}), partition=1, key='k4', timestamp_ms=1577836804004) producer.flush() time.sleep(1) - producer.send(topic='virt2_1', value=json.dumps({'value': 5}), partition=0, key='k5', timestamp_ms=1577836805000) - producer.send(topic='virt2_1', value=json.dumps({'value': 6}), partition=0, key='k6', timestamp_ms=1577836806000) + producer.send(topic='virt2_1', value=json.dumps({'value': 5}), partition=0, key='k5', timestamp_ms=1577836805005) + producer.send(topic='virt2_1', value=json.dumps({'value': 6}), partition=0, key='k6', timestamp_ms=1577836806006) producer.flush() time.sleep(1) - producer.send(topic='virt2_1', value=json.dumps({'value': 7}), partition=1, key='k7', timestamp_ms=1577836807000) - producer.send(topic='virt2_1', value=json.dumps({'value': 8}), partition=1, key='k8', timestamp_ms=1577836808000) + producer.send(topic='virt2_1', value=json.dumps({'value': 7}), partition=1, key='k7', timestamp_ms=1577836807007) + producer.send(topic='virt2_1', value=json.dumps({'value': 8}), partition=1, key='k8', timestamp_ms=1577836808008) producer.flush() time.sleep(10) @@ -869,14 +869,14 @@ def test_kafka_virtual_columns2(kafka_cluster): result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) expected = '''\ -1 k1 virt2_0 0 0 1577836801 -2 k2 virt2_0 0 1 1577836802 -3 k3 virt2_0 1 0 1577836803 -4 k4 virt2_0 1 1 1577836804 -5 k5 virt2_1 0 0 1577836805 -6 k6 virt2_1 0 1 1577836806 -7 k7 virt2_1 1 0 1577836807 -8 k8 virt2_1 1 1 1577836808 +1 k1 virt2_0 0 0 1577836801 1577836801001 +2 k2 virt2_0 0 1 1577836802 1577836802002 +3 k3 virt2_0 1 0 1577836803 1577836803003 +4 k4 virt2_0 1 1 1577836804 1577836804004 +5 k5 virt2_1 0 0 1577836805 1577836805005 +6 k6 virt2_1 0 1 1577836806 1577836806006 +7 k7 virt2_1 1 0 1577836807 1577836807007 +8 k8 virt2_1 1 1 1577836808 1577836808008 ''' assert TSV(result) == TSV(expected) From 16de65a6f0c1111d8e8cf8402d0d937552c111d1 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 28 May 2020 19:52:59 +0200 Subject: [PATCH 051/150] Add prefix & facility to librdkafka logs --- src/Storages/Kafka/StorageKafka.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 7731cf3c06a..16c424eb7be 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -324,10 +324,10 @@ void StorageKafka::updateConfiguration(cppkafka::Configuration & conf) } // No need to add any prefix, messages can be distinguished - conf.set_log_callback([this](cppkafka::KafkaHandleBase &, int level, const std::string & /* facility */, const std::string & message) + conf.set_log_callback([this](cppkafka::KafkaHandleBase &, int level, const std::string & facility, const std::string & message) { auto [poco_level, client_logs_level] = parseSyslogLevel(level); - LOG_IMPL(log, client_logs_level, poco_level, message); + LOG_IMPL(log, client_logs_level, poco_level, "[rdk:{}] {}", facility, message); }); // Configure interceptor to change thread name From 87b0ea26d38f89df2bdac938c2ecb7f91bdd2177 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 20:59:09 +0300 Subject: [PATCH 052/150] Try fix tests. --- src/DataStreams/BlocksBlockInputStream.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/DataStreams/BlocksBlockInputStream.h b/src/DataStreams/BlocksBlockInputStream.h index 6301a92b6a4..3062a24a44c 100644 --- a/src/DataStreams/BlocksBlockInputStream.h +++ b/src/DataStreams/BlocksBlockInputStream.h @@ -13,6 +13,7 @@ limitations under the License. */ #include #include +#include namespace DB @@ -38,7 +39,12 @@ protected: Block res = *it; ++it; - return Chunk(res.getColumns(), res.rows()); + + auto info = std::make_shared(); + info->bucket_num = res.info.bucket_num; + info->is_overflows = res.info.is_overflows; + + return Chunk(res.getColumns(), res.rows(), std::move(info)); } private: From b97ff8136ba17cb46e95bf521ffa561b9912be0a Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 28 May 2020 20:10:54 +0200 Subject: [PATCH 053/150] Make function static --- src/Storages/Kafka/StorageKafka.cpp | 2 +- src/Storages/Kafka/StorageKafka.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index ec0b7b91319..f69ac2686bc 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -152,7 +152,7 @@ StorageKafka::StorageKafka( task->deactivate(); } -const String StorageKafka::getDefaultClientId(const StorageID & table_id_) const +String StorageKafka::getDefaultClientId(const StorageID & table_id_) { std::stringstream ss; ss << VERSION_NAME << "-" << getFQDNOrHostName() << "-" << table_id_.database_name << "-" << table_id_.table_name; diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 977fe089792..1ea7d6dcad7 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -116,7 +116,7 @@ private: void updateConfiguration(cppkafka::Configuration & conf); void threadFunc(); - const String getDefaultClientId(const StorageID & table_id_) const; + static String getDefaultClientId(const StorageID & table_id_); bool streamToViews(); bool checkDependencies(const StorageID & table_id); }; From 7211af9404c47a527f74ae8f4c5c3c256f0d6319 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 28 May 2020 20:19:10 +0200 Subject: [PATCH 054/150] Fixing style --- src/Storages/Kafka/StorageKafka.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index c2d26dfa300..d1f350b02a6 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -37,7 +37,6 @@ #include - namespace DB { From a9c6fa1ca73dd2d35ee3d1152edd56d20b6db4e5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 May 2020 23:05:07 +0300 Subject: [PATCH 055/150] Try fix tests. --- src/Processors/Executors/PullingPipelineExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index af061a373cc..18722dfdc2f 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -76,8 +76,8 @@ void PullingPipelineExecutor::cancel() executor->cancel(); /// Read all data and finish execution. - Chunk chunk; - while (pull(chunk)); + /// Chunk chunk; + /// while (pull(chunk)); } Chunk PullingPipelineExecutor::getTotals() From 3cff070f897e812e8eb3f5f2dcc1d1a654f31771 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 28 May 2020 23:10:45 +0300 Subject: [PATCH 056/150] add context to IDatabase methods --- programs/copier/ClusterCopierApp.cpp | 2 +- programs/local/LocalServer.cpp | 10 +++++----- src/Databases/DatabaseAtomic.cpp | 6 +++--- src/Databases/DatabaseAtomic.h | 2 +- src/Databases/DatabaseDictionary.cpp | 8 ++++---- src/Databases/DatabaseDictionary.h | 8 ++++---- src/Databases/DatabaseFactory.cpp | 2 +- src/Databases/DatabaseLazy.cpp | 2 +- src/Databases/DatabaseLazy.h | 8 +++++--- src/Databases/DatabaseMemory.cpp | 8 ++++---- src/Databases/DatabaseMemory.h | 4 ++-- src/Databases/DatabaseMySQL.cpp | 12 ++++++------ src/Databases/DatabaseMySQL.h | 8 ++++---- src/Databases/DatabaseOnDisk.cpp | 11 +++++------ src/Databases/DatabaseOnDisk.h | 2 +- src/Databases/DatabaseWithDictionaries.cpp | 2 +- src/Databases/DatabasesCommon.cpp | 11 ++++++----- src/Databases/DatabasesCommon.h | 11 ++++++----- src/Databases/IDatabase.h | 16 ++++++++-------- src/Interpreters/ActionLocksManager.cpp | 8 ++++---- src/Interpreters/ActionLocksManager.h | 2 +- src/Interpreters/AsynchronousMetrics.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 8 ++++---- src/Interpreters/InterpreterCreateQuery.cpp | 6 +++--- src/Interpreters/InterpreterDropQuery.cpp | 2 +- src/Interpreters/InterpreterShowCreateQuery.cpp | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 6 +++--- src/Server/ReplicasStatusHandler.cpp | 2 +- src/Storages/StorageMerge.cpp | 12 ++++++------ src/Storages/StorageMerge.h | 2 +- src/Storages/System/StorageSystemColumns.cpp | 2 +- src/Storages/System/StorageSystemGraphite.cpp | 8 ++++---- src/Storages/System/StorageSystemMutations.cpp | 2 +- src/Storages/System/StorageSystemPartsBase.cpp | 2 +- src/Storages/System/StorageSystemReplicas.cpp | 2 +- .../System/StorageSystemReplicationQueue.cpp | 2 +- src/Storages/System/StorageSystemTables.cpp | 4 ++-- ...est_transform_query_for_external_database.cpp | 2 +- src/TableFunctions/TableFunctionMerge.cpp | 6 +++--- 39 files changed, 109 insertions(+), 106 deletions(-) diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index 9b1393204d4..ce4bf94589e 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -114,7 +114,7 @@ void ClusterCopierApp::mainImpl() registerDisks(); static const std::string default_database = "_local"; - DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database)); + DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, *context)); context->setCurrentDatabase(default_database); /// Initialize query scope just in case. diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 01acf250b1b..eb78c049825 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -118,13 +118,13 @@ void LocalServer::tryInitPath() } -static void attachSystemTables() +static void attachSystemTables(const Context & context) { DatabasePtr system_database = DatabaseCatalog::instance().tryGetDatabase(DatabaseCatalog::SYSTEM_DATABASE); if (!system_database) { /// TODO: add attachTableDelayed into DatabaseMemory to speedup loading - system_database = std::make_shared(DatabaseCatalog::SYSTEM_DATABASE); + system_database = std::make_shared(DatabaseCatalog::SYSTEM_DATABASE, context); DatabaseCatalog::instance().attachDatabase(DatabaseCatalog::SYSTEM_DATABASE, system_database); } @@ -202,7 +202,7 @@ try * if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons. */ std::string default_database = config().getString("default_database", "_local"); - DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database)); + DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, *context)); context->setCurrentDatabase(default_database); applyCmdOptions(); @@ -213,14 +213,14 @@ try LOG_DEBUG(log, "Loading metadata from {}", context->getPath()); loadMetadataSystem(*context); - attachSystemTables(); + attachSystemTables(*context); loadMetadata(*context); DatabaseCatalog::instance().loadDatabases(); LOG_DEBUG(log, "Loaded metadata."); } else { - attachSystemTables(); + attachSystemTables(*context); } processQueries(); diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 27a2441cec6..ed875f5fff4 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -288,15 +288,15 @@ void DatabaseAtomic::assertCanBeDetached(bool cleenup) "because some tables are still in use. Retry later.", ErrorCodes::DATABASE_NOT_EMPTY); } -DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const IDatabase::FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const Context & context, const IDatabase::FilterByNameFunction & filter_by_table_name) { - auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(filter_by_table_name); + auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(context, filter_by_table_name); return std::make_unique(std::move(typeid_cast(*base_iter))); } UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const { - if (auto table = tryGetTable(table_name)) + if (auto table = tryGetTable(table_name, global_context)) return table->getStorageID().uuid; return UUIDHelpers::Nil; } diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 84297a42843..71428fdb420 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -42,7 +42,7 @@ public: void drop(const Context & /*context*/) override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override; diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index 81bcbf7bc6b..c4c74f8e70c 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -50,18 +50,18 @@ Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_nam return tables; } -bool DatabaseDictionary::isTableExist(const String & table_name) const +bool DatabaseDictionary::isTableExist(const String & table_name, const Context &) const { return global_context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST; } -StoragePtr DatabaseDictionary::tryGetTable(const String & table_name) const +StoragePtr DatabaseDictionary::tryGetTable(const String & table_name, const Context &) const { auto load_result = global_context.getExternalDictionariesLoader().getLoadResult(table_name); return createStorageDictionary(getDatabaseName(), load_result); } -DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) { return std::make_unique(listTables(filter_by_table_name)); } @@ -71,7 +71,7 @@ bool DatabaseDictionary::empty() const return !global_context.getExternalDictionariesLoader().hasObjects(); } -ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const +ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const { String query; { diff --git a/src/Databases/DatabaseDictionary.h b/src/Databases/DatabaseDictionary.h index bf10dc134bd..b61c85033e8 100644 --- a/src/Databases/DatabaseDictionary.h +++ b/src/Databases/DatabaseDictionary.h @@ -29,11 +29,11 @@ public: return "Dictionary"; } - bool isTableExist(const String & table_name) const override; + bool isTableExist(const String & table_name, const Context & context) const override; - StoragePtr tryGetTable(const String & table_name) const override; + StoragePtr tryGetTable(const String & table_name, const Context & context) const override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; bool empty() const override; @@ -44,7 +44,7 @@ public: void shutdown() override; protected: - ASTPtr getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const override; + ASTPtr getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const override; private: mutable std::mutex mutex; diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 21feb122da1..f27bc509ebe 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -82,7 +82,7 @@ DatabasePtr DatabaseFactory::getImpl( else if (engine_name == "Atomic") return std::make_shared(database_name, metadata_path, context); else if (engine_name == "Memory") - return std::make_shared(database_name); + return std::make_shared(database_name, context); else if (engine_name == "Dictionary") return std::make_shared(database_name, context); diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 51ee664a42a..11e5272110e 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -132,7 +132,7 @@ StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const return loadTable(table_name); } -DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) { std::lock_guard lock(mutex); Strings filtered_tables; diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 4306e61b37b..2e24b687be5 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -51,13 +51,15 @@ public: time_t getObjectMetadataModificationTime(const String & table_name) const override; - bool isTableExist(const String & table_name) const override; + bool isTableExist(const String & table_name, const Context &) const override { return isTableExist(table_name); } + bool isTableExist(const String & table_name) const; - StoragePtr tryGetTable(const String & table_name) const override; + StoragePtr tryGetTable(const String & table_name, const Context &) const override { return tryGetTable(table_name); } + StoragePtr tryGetTable(const String & table_name) const; bool empty() const override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override; diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 84fec6bcc22..52b1f889943 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -16,8 +16,8 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -DatabaseMemory::DatabaseMemory(const String & name_) - : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")") +DatabaseMemory::DatabaseMemory(const String & name_, const Context & context) + : DatabaseWithOwnTablesBase(name_, "DatabaseMemory(" + name_ + ")", context) , data_path("data/" + escapeForFileName(database_name) + "/") {} @@ -64,7 +64,7 @@ ASTPtr DatabaseMemory::getCreateDatabaseQuery() const return create_query; } -ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const +ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const { std::lock_guard lock{mutex}; auto it = create_queries.find(table_name); @@ -80,7 +80,7 @@ ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, bool t UUID DatabaseMemory::tryGetTableUUID(const String & table_name) const { - if (auto table = tryGetTable(table_name)) + if (auto table = tryGetTable(table_name, global_context)) return table->getStorageID().uuid; return UUIDHelpers::Nil; } diff --git a/src/Databases/DatabaseMemory.h b/src/Databases/DatabaseMemory.h index ad34c4d9097..40cc808e42b 100644 --- a/src/Databases/DatabaseMemory.h +++ b/src/Databases/DatabaseMemory.h @@ -19,7 +19,7 @@ namespace DB class DatabaseMemory final : public DatabaseWithOwnTablesBase { public: - DatabaseMemory(const String & name_); + DatabaseMemory(const String & name_, const Context & context); String getEngineName() const override { return "Memory"; } @@ -34,7 +34,7 @@ public: const String & table_name, bool no_delay) override; - ASTPtr getCreateTableQueryImpl(const String & name, bool throw_on_error) const override; + ASTPtr getCreateTableQueryImpl(const String & name, const Context & context, bool throw_on_error) const override; ASTPtr getCreateDatabaseQuery() const override; /// DatabaseMemory allows to create tables, which store data on disk. diff --git a/src/Databases/DatabaseMySQL.cpp b/src/Databases/DatabaseMySQL.cpp index cbd3261975a..5d4b81014f9 100644 --- a/src/Databases/DatabaseMySQL.cpp +++ b/src/Databases/DatabaseMySQL.cpp @@ -89,7 +89,7 @@ bool DatabaseMySQL::empty() const return true; } -DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) { Tables tables; std::lock_guard lock(mutex); @@ -103,12 +103,12 @@ DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const FilterByNameFun return std::make_unique(tables); } -bool DatabaseMySQL::isTableExist(const String & name) const +bool DatabaseMySQL::isTableExist(const String & name, const Context &) const { - return bool(tryGetTable(name)); + return bool(tryGetTable(name, global_context)); } -StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name) const +StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name, const Context &) const { std::lock_guard lock(mutex); @@ -155,7 +155,7 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr return create_table_query; } -ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const +ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const { std::lock_guard lock(mutex); @@ -501,7 +501,7 @@ void DatabaseMySQL::createTable(const Context &, const String & table_name, cons /// XXX: hack /// In order to prevent users from broken the table structure by executing attach table database_name.table_name (...) /// we should compare the old and new create_query to make them completely consistent - const auto & origin_create_query = getCreateTableQuery(table_name); + const auto & origin_create_query = getCreateTableQuery(table_name, global_context); origin_create_query->as()->attach = true; if (queryToString(origin_create_query) != queryToString(create_query)) diff --git a/src/Databases/DatabaseMySQL.h b/src/Databases/DatabaseMySQL.h index a43da5d1762..70d8dc81e0f 100644 --- a/src/Databases/DatabaseMySQL.h +++ b/src/Databases/DatabaseMySQL.h @@ -32,13 +32,13 @@ public: bool empty() const override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; ASTPtr getCreateDatabaseQuery() const override; - bool isTableExist(const String & name) const override; + bool isTableExist(const String & name, const Context & context) const override; - StoragePtr tryGetTable(const String & name) const override; + StoragePtr tryGetTable(const String & name, const Context & context) const override; time_t getObjectMetadataModificationTime(const String & name) const override; @@ -59,7 +59,7 @@ public: void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; protected: - ASTPtr getCreateTableQueryImpl(const String & name, bool throw_on_error) const override; + ASTPtr getCreateTableQueryImpl(const String & name, const Context & context, bool throw_on_error) const override; private: const Context & global_context; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 70f86b19f1f..1b542c7a1ff 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -123,10 +123,9 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) } DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context) - : DatabaseWithOwnTablesBase(name, logger) + : DatabaseWithOwnTablesBase(name, logger, context) , metadata_path(metadata_path_) , data_path(data_path_) - , global_context(context.getGlobalContext()) { Poco::File(context.getPath() + data_path).createDirectories(); Poco::File(metadata_path).createDirectories(); @@ -160,7 +159,7 @@ void DatabaseOnDisk::createTable( throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); - if (isTableExist(table_name)) + if (isTableExist(table_name, global_context)) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); if (create.attach_short_syntax) @@ -267,7 +266,7 @@ void DatabaseOnDisk::renameTable( String table_metadata_path; ASTPtr attach_query; /// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case. - StoragePtr table = tryGetTable(table_name); + StoragePtr table = tryGetTable(table_name, global_context); detachTable(table_name); try { @@ -304,10 +303,10 @@ void DatabaseOnDisk::renameTable( Poco::File(table_metadata_path).remove(); } -ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const +ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, const Context &, bool throw_on_error) const { ASTPtr ast; - bool has_table = tryGetTable(table_name) != nullptr; + bool has_table = tryGetTable(table_name, global_context) != nullptr; auto table_metadata_path = getObjectMetadataPath(table_name); try { diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index 75609e231af..d4fb9b2aa17 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -76,6 +76,7 @@ protected: ASTPtr getCreateTableQueryImpl( const String & table_name, + const Context & context, bool throw_on_error) const override; ASTPtr getCreateQueryFromMetadata(const String & metadata_path, bool throw_on_error) const; @@ -85,7 +86,6 @@ protected: const String metadata_path; const String data_path; - const Context & global_context; }; } diff --git a/src/Databases/DatabaseWithDictionaries.cpp b/src/Databases/DatabaseWithDictionaries.cpp index 0d49078bd08..e0f2aa9286b 100644 --- a/src/Databases/DatabaseWithDictionaries.cpp +++ b/src/Databases/DatabaseWithDictionaries.cpp @@ -127,7 +127,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S "Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS); - if (isTableExist(dictionary_name)) + if (isTableExist(dictionary_name, global_context)) throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 5ad4fa20690..2a2ca1841cf 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -18,18 +19,18 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; } -DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger) - : IDatabase(name_), log(&Logger::get(logger)) +DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context) + : IDatabase(name_), log(&Logger::get(logger)), global_context(context.getGlobalContext()) { } -bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name) const +bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name, const Context &) const { std::lock_guard lock(mutex); return tables.find(table_name) != tables.end(); } -StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name) const +StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name, const Context &) const { std::lock_guard lock(mutex); auto it = tables.find(table_name); @@ -38,7 +39,7 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name) con return {}; } -DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const FilterByNameFunction & filter_by_table_name) +DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name) { std::lock_guard lock(mutex); if (!filter_by_table_name) diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index e195fc59ce9..4238fd30137 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -19,9 +19,9 @@ class Context; class DatabaseWithOwnTablesBase : public IDatabase { public: - bool isTableExist(const String & table_name) const override; + bool isTableExist(const String & table_name, const Context & context) const override; - StoragePtr tryGetTable(const String & table_name) const override; + StoragePtr tryGetTable(const String & table_name, const Context & context) const override; bool empty() const override; @@ -29,18 +29,19 @@ public: StoragePtr detachTable(const String & table_name) override; - DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; void shutdown() override; - virtual ~DatabaseWithOwnTablesBase() override; + ~DatabaseWithOwnTablesBase() override; protected: mutable std::mutex mutex; Tables tables; Poco::Logger * log; + const Context & global_context; - DatabaseWithOwnTablesBase(const String & name_, const String & logger); + DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context); void attachTableUnlocked(const String & table_name, const StoragePtr & table, std::unique_lock & lock); StoragePtr detachTableUnlocked(const String & table_name, std::unique_lock & lock); diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 24147a0e7c5..3d108cfd91c 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -130,7 +130,7 @@ public: virtual void loadStoredObjects(Context & /*context*/, bool /*has_force_restore_data_flag*/) {} /// Check the existence of the table. - virtual bool isTableExist(const String & name) const = 0; + virtual bool isTableExist(const String & name, const Context & context) const = 0; /// Check the existence of the dictionary virtual bool isDictionaryExist(const String & /*name*/) const @@ -139,7 +139,7 @@ public: } /// Get the table for work. Return nullptr if there is no table. - virtual StoragePtr tryGetTable(const String & name) const = 0; + virtual StoragePtr tryGetTable(const String & name, const Context & context) const = 0; virtual UUID tryGetTableUUID(const String & /*table_name*/) const { return UUIDHelpers::Nil; } @@ -147,7 +147,7 @@ public: /// Get an iterator that allows you to pass through all the tables. /// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above. - virtual DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name = {}) = 0; + virtual DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0; /// Get an iterator to pass through all the dictionaries. virtual DatabaseDictionariesIteratorPtr getDictionariesIterator([[maybe_unused]] const FilterByNameFunction & filter_by_dictionary_name = {}) @@ -249,14 +249,14 @@ public: } /// Get the CREATE TABLE query for the table. It can also provide information for detached tables for which there is metadata. - ASTPtr tryGetCreateTableQuery(const String & name) const noexcept + ASTPtr tryGetCreateTableQuery(const String & name, const Context & context) const noexcept { - return getCreateTableQueryImpl(name, false); + return getCreateTableQueryImpl(name, context, false); } - ASTPtr getCreateTableQuery(const String & name) const + ASTPtr getCreateTableQuery(const String & name, const Context & context) const { - return getCreateTableQueryImpl(name, true); + return getCreateTableQueryImpl(name, context, true); } /// Get the CREATE DICTIONARY query for the dictionary. Returns nullptr if dictionary doesn't exists. @@ -304,7 +304,7 @@ public: virtual ~IDatabase() {} protected: - virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, bool throw_on_error) const + virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, const Context & /*context*/, bool throw_on_error) const { if (throw_on_error) throw Exception("There is no SHOW CREATE TABLE query for Database" + getEngineName(), ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY); diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index fe5b3def658..8bc8542da3a 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -20,17 +20,17 @@ namespace ActionLocks template -inline void forEachTable(F && f) +inline void forEachTable(F && f, const Context & context) { for (auto & elem : DatabaseCatalog::instance().getDatabases()) - for (auto iterator = elem.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next()) f(iterator->table()); } -void ActionLocksManager::add(StorageActionBlockType action_type) +void ActionLocksManager::add(StorageActionBlockType action_type, const Context & context) { - forEachTable([&](const StoragePtr & table) { add(table, action_type); }); + forEachTable([&](const StoragePtr & table) { add(table, action_type); }, context); } void ActionLocksManager::add(const StorageID & table_id, StorageActionBlockType action_type) diff --git a/src/Interpreters/ActionLocksManager.h b/src/Interpreters/ActionLocksManager.h index 039a95ce218..50855676fd4 100644 --- a/src/Interpreters/ActionLocksManager.h +++ b/src/Interpreters/ActionLocksManager.h @@ -20,7 +20,7 @@ class ActionLocksManager { public: /// Adds new locks for each table - void add(StorageActionBlockType action_type); + void add(StorageActionBlockType action_type, const Context & context); /// Add new lock for a table if it has not been already added void add(const StorageID & table_id, StorageActionBlockType action_type); void add(const StoragePtr & table, StorageActionBlockType action_type); diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 59ff01bf972..a4e03d35d42 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -181,7 +181,7 @@ void AsynchronousMetrics::update() /// Lazy database can not contain MergeTree tables if (db.second->getEngineName() == "Lazy") continue; - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { ++total_number_of_tables; const auto & table = iterator->table(); diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 7606fdb255a..f292aaaf049 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -97,7 +97,7 @@ StorageID TemporaryTableHolder::getGlobalTableID() const StoragePtr TemporaryTableHolder::getTable() const { - auto table = temporary_tables->tryGetTable("_tmp_" + toString(id)); + auto table = temporary_tables->tryGetTable("_tmp_" + toString(id), *global_context); if (!table) throw Exception("Temporary table " + getGlobalTableID().getNameForLogs() + " not found", ErrorCodes::LOGICAL_ERROR); return table; @@ -108,7 +108,7 @@ void DatabaseCatalog::loadDatabases() { drop_delay_sec = global_context->getConfigRef().getInt("database_atomic_delay_before_drop_table_sec", default_drop_delay_sec); - auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE); + auto db_for_temporary_and_external_tables = std::make_shared(TEMPORARY_DATABASE, *global_context); attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables); loadMarkedAsDroppedTables(); @@ -206,7 +206,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( database = it->second; } - auto table = database->tryGetTable(table_id.table_name); + auto table = database->tryGetTable(table_id.table_name, *global_context); if (!table && exception) exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); @@ -331,7 +331,7 @@ bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id) const if (iter != databases.end()) db = iter->second; } - return db && db->isTableExist(table_id.table_name); + return db && db->isTableExist(table_id.table_name, *global_context); } void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id) const diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 147ef7d739b..bce15434418 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -504,7 +504,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const String as_database_name = context.resolveDatabase(create.as_database); String as_table_name = create.as_table; - ASTPtr as_create_ptr = DatabaseCatalog::instance().getDatabase(as_database_name)->getCreateTableQuery(as_table_name); + ASTPtr as_create_ptr = DatabaseCatalog::instance().getDatabase(as_database_name)->getCreateTableQuery(as_table_name, context); const auto & as_create = as_create_ptr->as(); const String qualified_name = backQuoteIfNeed(as_database_name) + "." + backQuoteIfNeed(as_table_name); @@ -546,7 +546,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) bool if_not_exists = create.if_not_exists; // Table SQL definition is available even if the table is detached - auto query = database->getCreateTableQuery(create.table); + auto query = database->getCreateTableQuery(create.table, context); create = query->as(); // Copy the saved create query, but use ATTACH instead of CREATE create.attach = true; create.attach_short_syntax = true; @@ -608,7 +608,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, guard = DatabaseCatalog::instance().getDDLGuard(create.database, table_name); /// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard. - if (database->isTableExist(table_name)) + if (database->isTableExist(table_name, context)) { /// TODO Check structure of table if (create.if_not_exists) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ecdad0168f0..8015e28d467 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -234,7 +234,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS ASTDropQuery query; query.kind = kind; query.database = database_name; - for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { query.table = iterator->name(); executeToTable({query.database, query.table}, query); diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 9938910cf1e..30005c7b169 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -50,7 +50,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl() auto resolve_table_type = show_query->temporary ? Context::ResolveExternal : Context::ResolveOrdinary; auto table_id = context.resolveStorageID(*show_query, resolve_table_type); context.checkAccess(AccessType::SHOW_COLUMNS, table_id); - create_query = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name); + create_query = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name, context); } else if ((show_query = query_ptr->as())) { diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index f3d48651148..557fb4c71e8 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -144,7 +144,7 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type, auto access = context.getAccess(); for (auto & elem : DatabaseCatalog::instance().getDatabases()) { - for (auto iterator = elem.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!access->isGranted(log, getRequiredAccessType(action_type), elem.first, iterator->name())) continue; @@ -332,7 +332,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, { /// If table was already dropped by anyone, an exception will be thrown auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - create_ast = database->getCreateTableQuery(replica.table_name); + create_ast = database->getCreateTableQuery(replica.table_name, context); database->detachTable(replica.table_name); } @@ -369,7 +369,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) for (auto & elem : catalog.getDatabases()) { DatabasePtr & database = elem.second; - for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (dynamic_cast(iterator->table().get())) replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name()}); diff --git a/src/Server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp index f2d1ffe2ee5..986af59d3a4 100644 --- a/src/Server/ReplicasStatusHandler.cpp +++ b/src/Server/ReplicasStatusHandler.cpp @@ -44,7 +44,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request if (db.second->getEngineName() == "Lazy") continue; - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { const auto & table = iterator->table(); StorageReplicatedMergeTree * table_replicated = dynamic_cast(table.get()); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index fb6d88c8d33..6f76ae4b7b3 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -62,7 +62,7 @@ StorageMerge::StorageMerge( template StoragePtr StorageMerge::getFirstTable(F && predicate) const { - auto iterator = getDatabaseIterator(); + auto iterator = getDatabaseIterator(global_context); while (iterator->isValid()) { @@ -110,7 +110,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage(const Context & { auto stage_in_source_tables = QueryProcessingStage::FetchColumns; - DatabaseTablesIteratorPtr iterator = getDatabaseIterator(); + DatabaseTablesIteratorPtr iterator = getDatabaseIterator(context); size_t selected_table_size = 0; @@ -329,7 +329,7 @@ Pipes StorageMerge::createSources(const SelectQueryInfo & query_info, const Quer StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String & query_id, const Settings & settings) const { StorageListWithLocks selected_tables; - auto iterator = getDatabaseIterator(); + auto iterator = getDatabaseIterator(global_context); while (iterator->isValid()) { @@ -349,7 +349,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( const ASTPtr & query, bool has_virtual_column, const String & query_id, const Settings & settings) const { StorageListWithLocks selected_tables; - DatabaseTablesIteratorPtr iterator = getDatabaseIterator(); + DatabaseTablesIteratorPtr iterator = getDatabaseIterator(global_context); auto virtual_column = ColumnString::create(); @@ -384,12 +384,12 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( } -DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator() const +DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const Context & context) const { checkStackSize(); auto database = DatabaseCatalog::instance().getDatabase(source_database); auto table_name_match = [this](const String & table_name_) { return table_name_regexp.match(table_name_); }; - return database->getTablesIterator(table_name_match); + return database->getTablesIterator(context, table_name_match); } diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index ebe40109d84..401c5facf0c 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -61,7 +61,7 @@ private: template StoragePtr getFirstTable(F && predicate) const; - DatabaseTablesIteratorPtr getDatabaseIterator() const; + DatabaseTablesIteratorPtr getDatabaseIterator(const Context & context) const; NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index ab824fc8bdc..8eb8856512e 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -301,7 +301,7 @@ Pipes StorageSystemColumns::read( const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { const String & table_name = iterator->name(); storages.emplace(std::piecewise_construct, diff --git a/src/Storages/System/StorageSystemGraphite.cpp b/src/Storages/System/StorageSystemGraphite.cpp index cde75aa5550..bfa25a99838 100644 --- a/src/Storages/System/StorageSystemGraphite.cpp +++ b/src/Storages/System/StorageSystemGraphite.cpp @@ -25,7 +25,7 @@ NamesAndTypesList StorageSystemGraphite::getNamesAndTypes() /* * Looking for (Replicated)*GraphiteMergeTree and get all configuration parameters for them */ -static StorageSystemGraphite::Configs getConfigs() +static StorageSystemGraphite::Configs getConfigs(const Context & context) { const Databases databases = DatabaseCatalog::instance().getDatabases(); StorageSystemGraphite::Configs graphite_configs; @@ -36,7 +36,7 @@ static StorageSystemGraphite::Configs getConfigs() if (db.second->getEngineName() == "Lazy") continue; - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { const auto & table = iterator->table(); @@ -71,9 +71,9 @@ static StorageSystemGraphite::Configs getConfigs() return graphite_configs; } -void StorageSystemGraphite::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const +void StorageSystemGraphite::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const { - Configs graphite_configs = getConfigs(); + Configs graphite_configs = getConfigs(context); for (const auto & config : graphite_configs) { diff --git a/src/Storages/System/StorageSystemMutations.cpp b/src/Storages/System/StorageSystemMutations.cpp index d4a262860dc..685565d82e1 100644 --- a/src/Storages/System/StorageSystemMutations.cpp +++ b/src/Storages/System/StorageSystemMutations.cpp @@ -51,7 +51,7 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 6356e6d699e..3b97cbb2d9b 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -111,7 +111,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const const DatabasePtr database = databases.at(database_name); offsets[i] = i ? offsets[i - 1] : 0; - for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next()) { String table_name = iterator->name(); StoragePtr storage = iterator->table(); diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 734d0098d8d..251b45e44b6 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -76,7 +76,7 @@ Pipes StorageSystemReplicas::read( if (db.second->getEngineName() == "Lazy") continue; const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; diff --git a/src/Storages/System/StorageSystemReplicationQueue.cpp b/src/Storages/System/StorageSystemReplicationQueue.cpp index a6f0af8cc6c..2c188cf3734 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -60,7 +60,7 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first); - for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next()) + for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next()) { if (!dynamic_cast(iterator->table().get())) continue; diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 2caf599fe51..2bf6595bf53 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -232,7 +232,7 @@ protected: const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name); if (!tables_it || !tables_it->isValid()) - tables_it = database->getTablesIterator(); + tables_it = database->getTablesIterator(context); const bool need_lock_structure = needLockStructure(database, getPort().getHeader()); @@ -331,7 +331,7 @@ protected: if (columns_mask[src_index] || columns_mask[src_index + 1]) { - ASTPtr ast = database->tryGetCreateTableQuery(table_name); + ASTPtr ast = database->tryGetCreateTableQuery(table_name, context); if (columns_mask[src_index++]) res_columns[res_index++]->insert(ast ? queryToString(ast) : ""); diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index df6ee2bd0a5..480852f9b6b 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -30,7 +30,7 @@ struct State explicit State(Context & context_) : context(context_) { registerFunctions(); - DatabasePtr database = std::make_shared("test"); + DatabasePtr database = std::make_shared("test", context); database->attachTable("table", StorageMemory::create(StorageID("test", "table"), ColumnsDescription{columns}, ConstraintsDescription{})); context.makeGlobalContext(); DatabaseCatalog::instance().attachDatabase("test", database); diff --git a/src/TableFunctions/TableFunctionMerge.cpp b/src/TableFunctions/TableFunctionMerge.cpp index 7a161c13bdc..cd924270f7c 100644 --- a/src/TableFunctions/TableFunctionMerge.cpp +++ b/src/TableFunctions/TableFunctionMerge.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes } -static NamesAndTypesList chooseColumns(const String & source_database, const String & table_name_regexp_) +static NamesAndTypesList chooseColumns(const String & source_database, const String & table_name_regexp_, const Context & context) { OptimizedRegularExpression table_name_regexp(table_name_regexp_); auto table_name_match = [&](const String & table_name) { return table_name_regexp.match(table_name); }; @@ -31,7 +31,7 @@ static NamesAndTypesList chooseColumns(const String & source_database, const Str { auto database = DatabaseCatalog::instance().getDatabase(source_database); - auto iterator = database->getTablesIterator(table_name_match); + auto iterator = database->getTablesIterator(context, table_name_match); if (iterator->isValid()) any_table = iterator->table(); @@ -69,7 +69,7 @@ StoragePtr TableFunctionMerge::executeImpl(const ASTPtr & ast_function, const Co auto res = StorageMerge::create( StorageID(getDatabaseName(), table_name), - ColumnsDescription{chooseColumns(source_database, table_name_regexp)}, + ColumnsDescription{chooseColumns(source_database, table_name_regexp, context)}, source_database, table_name_regexp, context); From 03430b1763733644fd7b77f542a3c5bd1aeee6f3 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 28 May 2020 23:11:29 +0200 Subject: [PATCH 057/150] Change the test, accept new librdkafka behaviour --- tests/integration/test_storage_kafka/test.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 13577864870..d09cc5659a1 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -1475,6 +1475,7 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): # as it's a bit tricky to hit the proper moment - let's check in logs if we did it correctly assert instance.contains_in_log("Local: Waiting for coordinator") + assert instance.contains_in_log("All commit attempts failed") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') print(result) @@ -1484,7 +1485,10 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): DROP TABLE test.view; ''') - assert TSV(result) == TSV('22\t22\t22') + # After https://github.com/edenhill/librdkafka/issues/2631 + # timeout triggers rebalance, making further commits to the topic after getting back online + # impossible. So we have a duplicate in that scenario, but we report that situation properly. + assert TSV(result) == TSV('42\t22\t22') From 86959ca97f36217c60f3c5b8a86161793736e460 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 01:23:29 +0300 Subject: [PATCH 058/150] Addition to #11184 --- tests/config/log_queries.xml | 7 ------- 1 file changed, 7 deletions(-) delete mode 100644 tests/config/log_queries.xml diff --git a/tests/config/log_queries.xml b/tests/config/log_queries.xml deleted file mode 100644 index 25261072ade..00000000000 --- a/tests/config/log_queries.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - 1 - - - From c071e0a473c39d8e48eac154037f96ad56557095 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 29 May 2020 02:01:18 +0300 Subject: [PATCH 059/150] add context to DatabaseCatalog methods --- .../InputStreamFromASTInsertQuery.cpp | 2 +- .../PushingToViewsBlockOutputStream.cpp | 2 +- src/DataStreams/tests/union_stream2.cpp | 2 +- src/Functions/FunctionJoinGet.cpp | 2 +- src/Functions/hasColumnInTable.cpp | 2 +- src/Interpreters/ActionLocksManager.cpp | 9 +++++-- src/Interpreters/ActionLocksManager.h | 3 +++ src/Interpreters/ActionsVisitor.cpp | 2 +- .../ClusterProxy/SelectStreamFactory.cpp | 2 +- src/Interpreters/Context.cpp | 2 +- src/Interpreters/DDLWorker.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 27 ++++++++++--------- src/Interpreters/DatabaseCatalog.h | 16 ++++++----- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- .../InJoinSubqueriesPreprocessor.cpp | 2 +- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterCheckQuery.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- src/Interpreters/InterpreterDropQuery.cpp | 6 ++--- src/Interpreters/InterpreterExistsQuery.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- .../InterpreterKillQueryQuery.cpp | 2 +- src/Interpreters/InterpreterOptimizeQuery.cpp | 2 +- src/Interpreters/InterpreterRenameQuery.cpp | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 6 ++--- src/Interpreters/InterpreterWatchQuery.cpp | 2 +- src/Interpreters/JoinedTables.cpp | 4 +-- src/Interpreters/SystemLog.h | 6 ++--- src/Interpreters/getTableExpressions.cpp | 2 +- src/Interpreters/interpretSubquery.cpp | 2 +- src/Server/MySQLHandler.cpp | 2 +- src/Server/TCPHandler.cpp | 6 ++--- src/Storages/Kafka/StorageKafka.cpp | 4 +-- src/Storages/LiveView/StorageLiveView.cpp | 2 +- src/Storages/LiveView/StorageLiveView.h | 2 +- src/Storages/StorageBuffer.cpp | 12 ++++----- src/Storages/StorageBuffer.h | 2 +- src/Storages/StorageMaterializedView.cpp | 8 +++--- src/Storages/StorageMergeTree.cpp | 4 +-- src/Storages/StorageReplicatedMergeTree.cpp | 6 ++--- src/Storages/getStructureOfRemoteTable.cpp | 2 +- 43 files changed, 93 insertions(+), 82 deletions(-) diff --git a/src/DataStreams/InputStreamFromASTInsertQuery.cpp b/src/DataStreams/InputStreamFromASTInsertQuery.cpp index 3b4946e4bc9..47b61294da3 100644 --- a/src/DataStreams/InputStreamFromASTInsertQuery.cpp +++ b/src/DataStreams/InputStreamFromASTInsertQuery.cpp @@ -58,7 +58,7 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery( if (context.getSettingsRef().input_format_defaults_for_omitted_fields && ast_insert_query->table_id && !input_function) { - StoragePtr storage = DatabaseCatalog::instance().getTable(ast_insert_query->table_id); + StoragePtr storage = DatabaseCatalog::instance().getTable(ast_insert_query->table_id, context); auto column_defaults = storage->getColumns().getDefaults(); if (!column_defaults.empty()) res_stream = std::make_shared(res_stream, column_defaults, context); diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 7f730b5fd3f..f68e0d81c03 100644 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -59,7 +59,7 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( for (const auto & database_table : dependencies) { - auto dependent_table = DatabaseCatalog::instance().getTable(database_table); + auto dependent_table = DatabaseCatalog::instance().getTable(database_table, context); ASTPtr query; BlockOutputStreamPtr out; diff --git a/src/DataStreams/tests/union_stream2.cpp b/src/DataStreams/tests/union_stream2.cpp index 6e0997e7f72..5b84d89a435 100644 --- a/src/DataStreams/tests/union_stream2.cpp +++ b/src/DataStreams/tests/union_stream2.cpp @@ -35,7 +35,7 @@ try Names column_names; column_names.push_back("WatchID"); - StoragePtr table = DatabaseCatalog::instance().getTable({"default", "hits6"}); + StoragePtr table = DatabaseCatalog::instance().getTable({"default", "hits6"}, context); QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); auto pipes = table->read(column_names, {}, context, stage, settings.max_block_size, settings.max_threads); diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index d6be2469759..a4569684e7f 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -43,7 +43,7 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co ++dot; } String table_name = join_name.substr(dot); - auto table = DatabaseCatalog::instance().getTable({database_name, table_name}); + auto table = DatabaseCatalog::instance().getTable({database_name, table_name}, context); auto storage_join = std::dynamic_pointer_cast(table); if (!storage_join) throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; diff --git a/src/Functions/hasColumnInTable.cpp b/src/Functions/hasColumnInTable.cpp index 03e5a4513db..b9ec2b84837 100644 --- a/src/Functions/hasColumnInTable.cpp +++ b/src/Functions/hasColumnInTable.cpp @@ -113,7 +113,7 @@ void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers & bool has_column; if (host_name.empty()) { - const StoragePtr & table = DatabaseCatalog::instance().getTable({database_name, table_name}); + const StoragePtr & table = DatabaseCatalog::instance().getTable({database_name, table_name}, global_context); has_column = table->getColumns().hasPhysical(column_name); } else diff --git a/src/Interpreters/ActionLocksManager.cpp b/src/Interpreters/ActionLocksManager.cpp index 8bc8542da3a..e8887f44a45 100644 --- a/src/Interpreters/ActionLocksManager.cpp +++ b/src/Interpreters/ActionLocksManager.cpp @@ -19,6 +19,11 @@ namespace ActionLocks } +ActionLocksManager::ActionLocksManager(const Context & context) + : global_context(context.getGlobalContext()) +{ +} + template inline void forEachTable(F && f, const Context & context) { @@ -35,7 +40,7 @@ void ActionLocksManager::add(StorageActionBlockType action_type, const Context & void ActionLocksManager::add(const StorageID & table_id, StorageActionBlockType action_type) { - if (auto table = DatabaseCatalog::instance().tryGetTable(table_id)) + if (auto table = DatabaseCatalog::instance().tryGetTable(table_id, global_context)) add(table, action_type); } @@ -60,7 +65,7 @@ void ActionLocksManager::remove(StorageActionBlockType action_type) void ActionLocksManager::remove(const StorageID & table_id, StorageActionBlockType action_type) { - if (auto table = DatabaseCatalog::instance().tryGetTable(table_id)) + if (auto table = DatabaseCatalog::instance().tryGetTable(table_id, global_context)) remove(table, action_type); } diff --git a/src/Interpreters/ActionLocksManager.h b/src/Interpreters/ActionLocksManager.h index 50855676fd4..ea5d96ad8ce 100644 --- a/src/Interpreters/ActionLocksManager.h +++ b/src/Interpreters/ActionLocksManager.h @@ -19,6 +19,8 @@ class Context; class ActionLocksManager { public: + ActionLocksManager(const Context & context); + /// Adds new locks for each table void add(StorageActionBlockType action_type, const Context & context); /// Add new lock for a table if it has not been already added @@ -41,6 +43,7 @@ private: mutable std::mutex mutex; StorageLocks storage_locks; + const Context & global_context; }; } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 7c2133e629f..6b227657ba5 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -670,7 +670,7 @@ SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_su if (identifier) { auto table_id = data.context.resolveStorageID(right_in_operand); - StoragePtr table = DatabaseCatalog::instance().tryGetTable(table_id); + StoragePtr table = DatabaseCatalog::instance().tryGetTable(table_id, data.context); if (table) { diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 53c61bea60e..d601d460be9 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -191,7 +191,7 @@ void SelectStreamFactory::createForShard( else { auto resolved_id = context.resolveStorageID(main_table); - main_table_storage = DatabaseCatalog::instance().tryGetTable(resolved_id); + main_table_storage = DatabaseCatalog::instance().tryGetTable(resolved_id, context); } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 4605c606594..9c78fe59da1 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2017,7 +2017,7 @@ std::shared_ptr Context::getActionLocksManager() auto lock = getLock(); if (!shared->action_locks_manager) - shared->action_locks_manager = std::make_shared(); + shared->action_locks_manager = std::make_shared(*this); return shared->action_locks_manager; } diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index fefeeeed988..ee91d903416 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -634,7 +634,7 @@ void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper) { /// It's not CREATE DATABASE auto table_id = context.tryResolveStorageID(*query_with_table, Context::ResolveOrdinary); - storage = DatabaseCatalog::instance().tryGetTable(table_id); + storage = DatabaseCatalog::instance().tryGetTable(table_id, context); } /// For some reason we check consistency of cluster definition only diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index f292aaaf049..a2157641d4d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -159,6 +159,7 @@ DatabaseAndTable DatabaseCatalog::tryGetByUUID(const UUID & uuid) const DatabaseAndTable DatabaseCatalog::getTableImpl( const StorageID & table_id, + const Context & context, std::optional * exception) const { if (!table_id) @@ -206,7 +207,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( database = it->second; } - auto table = database->tryGetTable(table_id.table_name, *global_context); + auto table = database->tryGetTable(table_id.table_name, context); if (!table && exception) exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); @@ -319,7 +320,7 @@ Databases DatabaseCatalog::getDatabases() const return databases; } -bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id) const +bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id, const Context & context) const { if (table_id.hasUUID()) return tryGetByUUID(table_id.uuid).second != nullptr; @@ -331,12 +332,12 @@ bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id) const if (iter != databases.end()) db = iter->second; } - return db && db->isTableExist(table_id.table_name, *global_context); + return db && db->isTableExist(table_id.table_name, context); } -void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id) const +void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id, const Context & context) const { - if (isTableExist(table_id)) + if (isTableExist(table_id, context)) throw Exception("Table " + table_id.getNameForLogs() + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); } @@ -468,32 +469,32 @@ bool DatabaseCatalog::isDictionaryExist(const StorageID & table_id) const return db && db->isDictionaryExist(table_id.getTableName()); } -StoragePtr DatabaseCatalog::getTable(const StorageID & table_id) const +StoragePtr DatabaseCatalog::getTable(const StorageID & table_id, const Context & context) const { std::optional exc; - auto res = getTableImpl(table_id, &exc); + auto res = getTableImpl(table_id, context, &exc); if (!res.second) throw Exception(*exc); return res.second; } -StoragePtr DatabaseCatalog::tryGetTable(const StorageID & table_id) const +StoragePtr DatabaseCatalog::tryGetTable(const StorageID & table_id, const Context & context) const { - return getTableImpl(table_id, nullptr).second; + return getTableImpl(table_id, context, nullptr).second; } -DatabaseAndTable DatabaseCatalog::getDatabaseAndTable(const StorageID & table_id) const +DatabaseAndTable DatabaseCatalog::getDatabaseAndTable(const StorageID & table_id, const Context & context) const { std::optional exc; - auto res = getTableImpl(table_id, &exc); + auto res = getTableImpl(table_id, context, &exc); if (!res.second) throw Exception(*exc); return res; } -DatabaseAndTable DatabaseCatalog::tryGetDatabaseAndTable(const StorageID & table_id) const +DatabaseAndTable DatabaseCatalog::tryGetDatabaseAndTable(const StorageID & table_id, const Context & context) const { - return getTableImpl(table_id, nullptr); + return getTableImpl(table_id, context, nullptr); } void DatabaseCatalog::loadMarkedAsDroppedTables() diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index a481e3d7e5e..ac8a99c58d1 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -129,15 +129,17 @@ public: DatabasePtr getDatabase(const String & database_name, const Context & local_context) const; /// For all of the following methods database_name in table_id must be not empty (even for temporary tables). - void assertTableDoesntExist(const StorageID & table_id) const; - bool isTableExist(const StorageID & table_id) const; + void assertTableDoesntExist(const StorageID & table_id, const Context & context) const; + bool isTableExist(const StorageID & table_id, const Context & context) const; bool isDictionaryExist(const StorageID & table_id) const; - StoragePtr getTable(const StorageID & table_id) const; - StoragePtr tryGetTable(const StorageID & table_id) const; - DatabaseAndTable getDatabaseAndTable(const StorageID & table_id) const; - DatabaseAndTable tryGetDatabaseAndTable(const StorageID & table_id) const; - DatabaseAndTable getTableImpl(const StorageID & table_id, std::optional * exception = nullptr) const; + StoragePtr getTable(const StorageID & table_id, const Context & context) const; + StoragePtr tryGetTable(const StorageID & table_id, const Context & context) const; + DatabaseAndTable getDatabaseAndTable(const StorageID & table_id, const Context & context) const; + DatabaseAndTable tryGetDatabaseAndTable(const StorageID & table_id, const Context & context) const; + DatabaseAndTable getTableImpl(const StorageID & table_id, + const Context & context, + std::optional * exception = nullptr) const; void addDependency(const StorageID & from, const StorageID & where); void removeDependency(const StorageID & from, const StorageID & where); diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b38af6feef9..6f846cc0277 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -322,7 +322,7 @@ SetPtr SelectQueryExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & if (!table) return nullptr; auto table_id = context.resolveStorageID(subquery_or_table_name); - const auto storage = DatabaseCatalog::instance().getTable(table_id); + const auto storage = DatabaseCatalog::instance().getTable(table_id, context); if (storage->getName() != "Set") return nullptr; const auto storage_set = std::dynamic_pointer_cast(storage); diff --git a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index f40e91e7dcd..432a7f40b12 100644 --- a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -27,7 +27,7 @@ namespace StoragePtr tryGetTable(const ASTPtr & database_and_table, const Context & context) { auto table_id = context.resolveStorageID(database_and_table); - return DatabaseCatalog::instance().tryGetTable(table_id); + return DatabaseCatalog::instance().tryGetTable(table_id, context); } using CheckShardsAndTables = InJoinSubqueriesPreprocessor::CheckShardsAndTables; diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index c2ab8776c25..205f4afb793 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -42,7 +42,7 @@ BlockIO InterpreterAlterQuery::execute() context.checkAccess(getRequiredAccess()); auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); /// Add default database to table identifiers that we can encounter in e.g. default expressions, /// mutation expression, etc. diff --git a/src/Interpreters/InterpreterCheckQuery.cpp b/src/Interpreters/InterpreterCheckQuery.cpp index b8f7203e607..b3cd807abe5 100644 --- a/src/Interpreters/InterpreterCheckQuery.cpp +++ b/src/Interpreters/InterpreterCheckQuery.cpp @@ -41,7 +41,7 @@ BlockIO InterpreterCheckQuery::execute() auto table_id = context.resolveStorageID(check, Context::ResolveOrdinary); context.checkAccess(AccessType::SHOW_TABLES, table_id); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); auto check_results = table->checkData(query_ptr, context); Block block; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index bce15434418..488fd79199d 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -406,7 +406,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS else if (!create.as_table.empty()) { String as_database_name = context.resolveDatabase(create.as_database); - StoragePtr as_storage = DatabaseCatalog::instance().getTable({as_database_name, create.as_table}); + StoragePtr as_storage = DatabaseCatalog::instance().getTable({as_database_name, create.as_table}, context); /// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table. as_storage_lock = as_storage->lockStructureForShare( diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 1a6dbbcfc4d..d457fefed6a 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -86,7 +86,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl() { auto table_id = context.resolveStorageID(table_expression.database_and_table_name); context.checkAccess(AccessType::SHOW_COLUMNS, table_id); - table = DatabaseCatalog::instance().getTable(table_id); + table = DatabaseCatalog::instance().getTable(table_id, context); } auto table_lock = table->lockStructureForShare( diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 8015e28d467..5ffce2fc3ec 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -81,8 +81,8 @@ BlockIO InterpreterDropQuery::executeToTable( auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); /// If table was already dropped by anyone, an exception will be thrown - auto [database, table] = query.if_exists ? DatabaseCatalog::instance().tryGetDatabaseAndTable(table_id) - : DatabaseCatalog::instance().getDatabaseAndTable(table_id); + auto [database, table] = query.if_exists ? DatabaseCatalog::instance().tryGetDatabaseAndTable(table_id, context) + : DatabaseCatalog::instance().getDatabaseAndTable(table_id, context); if (database && table) { @@ -182,7 +182,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name, auto resolved_id = context_handle.tryResolveStorageID(StorageID("", table_name), Context::ResolveExternal); if (resolved_id) { - StoragePtr table = DatabaseCatalog::instance().getTable(resolved_id); + StoragePtr table = DatabaseCatalog::instance().getTable(resolved_id, context); if (kind == ASTDropQuery::Kind::Truncate) { auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); diff --git a/src/Interpreters/InterpreterExistsQuery.cpp b/src/Interpreters/InterpreterExistsQuery.cpp index 993b3631e06..8539cb6816e 100644 --- a/src/Interpreters/InterpreterExistsQuery.cpp +++ b/src/Interpreters/InterpreterExistsQuery.cpp @@ -50,7 +50,7 @@ BlockInputStreamPtr InterpreterExistsQuery::executeImpl() { String database = context.resolveDatabase(exists_query->database); context.checkAccess(AccessType::SHOW_TABLES, database, exists_query->table); - result = DatabaseCatalog::instance().isTableExist({database, exists_query->table}); + result = DatabaseCatalog::instance().isTableExist({database, exists_query->table}, context); } } else if ((exists_query = query_ptr->as())) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 1c1e21fc32c..dacd7ca5f20 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -76,7 +76,7 @@ namespace if (const auto * identifier = expression.database_and_table_name->as()) { auto table_id = data.context.resolveStorageID(*identifier); - const auto & storage = DatabaseCatalog::instance().getTable(table_id); + const auto & storage = DatabaseCatalog::instance().getTable(table_id, data.context); if (auto * storage_view = dynamic_cast(storage.get())) storage_view->getRuntimeViewQuery(&select_query, data.context, true); diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b6efa5d6d46..069dfbf006f 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -70,7 +70,7 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query) } query.table_id = context.resolveStorageID(query.table_id); - return DatabaseCatalog::instance().getTable(query.table_id); + return DatabaseCatalog::instance().getTable(query.table_id, context); } Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) const diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 42afd0ef477..82c134aeba6 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -261,7 +261,7 @@ BlockIO InterpreterKillQueryQuery::execute() CancellationCode code = CancellationCode::Unknown; if (!query.test) { - auto storage = DatabaseCatalog::instance().tryGetTable(table_id); + auto storage = DatabaseCatalog::instance().tryGetTable(table_id, context); if (!storage) code = CancellationCode::NotFound; else diff --git a/src/Interpreters/InterpreterOptimizeQuery.cpp b/src/Interpreters/InterpreterOptimizeQuery.cpp index b6c50e59cc3..c47fe1160cf 100644 --- a/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -25,7 +25,7 @@ BlockIO InterpreterOptimizeQuery::execute() context.checkAccess(getRequiredAccess()); auto table_id = context.resolveStorageID(ast, Context::ResolveOrdinary); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context); return {}; } diff --git a/src/Interpreters/InterpreterRenameQuery.cpp b/src/Interpreters/InterpreterRenameQuery.cpp index 78d1c7ee486..de2b6bb0c1c 100644 --- a/src/Interpreters/InterpreterRenameQuery.cpp +++ b/src/Interpreters/InterpreterRenameQuery.cpp @@ -78,7 +78,7 @@ BlockIO InterpreterRenameQuery::execute() for (auto & elem : descriptions) { if (!rename.exchange) - database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name)); + database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name), context); database_catalog.getDatabase(elem.from_database_name)->renameTable( context, diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 557fb4c71e8..fedda7cab50 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -321,7 +321,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, context.checkAccess(AccessType::SYSTEM_RESTART_REPLICA, replica); auto table_ddl_guard = need_ddl_guard ? DatabaseCatalog::instance().getDDLGuard(replica.getDatabaseName(), replica.getTableName()) : nullptr; - auto [database, table] = DatabaseCatalog::instance().tryGetDatabaseAndTable(replica); + auto [database, table] = DatabaseCatalog::instance().tryGetDatabaseAndTable(replica, context); ASTPtr create_ast; /// Detach actions @@ -394,7 +394,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context) void InterpreterSystemQuery::syncReplica(ASTSystemQuery &) { context.checkAccess(AccessType::SYSTEM_SYNC_REPLICA, table_id); - StoragePtr table = DatabaseCatalog::instance().getTable(table_id); + StoragePtr table = DatabaseCatalog::instance().getTable(table_id, context); if (auto * storage_replicated = dynamic_cast(table.get())) { @@ -416,7 +416,7 @@ void InterpreterSystemQuery::flushDistributed(ASTSystemQuery &) { context.checkAccess(AccessType::SYSTEM_FLUSH_DISTRIBUTED, table_id); - if (auto * storage_distributed = dynamic_cast(DatabaseCatalog::instance().getTable(table_id).get())) + if (auto * storage_distributed = dynamic_cast(DatabaseCatalog::instance().getTable(table_id, context).get())) storage_distributed->flushClusterNodesAllData(); else throw Exception("Table " + table_id.getNameForLogs() + " is not distributed", ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Interpreters/InterpreterWatchQuery.cpp b/src/Interpreters/InterpreterWatchQuery.cpp index dc1ae6a7cad..489be488b4d 100644 --- a/src/Interpreters/InterpreterWatchQuery.cpp +++ b/src/Interpreters/InterpreterWatchQuery.cpp @@ -40,7 +40,7 @@ BlockIO InterpreterWatchQuery::execute() auto table_id = context.resolveStorageID(query, Context::ResolveOrdinary); /// Get storage - storage = DatabaseCatalog::instance().tryGetTable(table_id); + storage = DatabaseCatalog::instance().tryGetTable(table_id, context); if (!storage) throw Exception("Table " + table_id.getNameForLogs() + " doesn't exist.", diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index 3bd883dcce8..7450890952a 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -181,7 +181,7 @@ StoragePtr JoinedTables::getLeftTableStorage() } /// Read from table. Even without table expression (implicit SELECT ... FROM system.one). - return DatabaseCatalog::instance().getTable(table_id); + return DatabaseCatalog::instance().getTable(table_id, context); } bool JoinedTables::resolveTables() @@ -261,7 +261,7 @@ std::shared_ptr JoinedTables::makeTableJoin(const ASTSelectQuery & se if (table_to_join.database_and_table_name) { auto joined_table_id = context.resolveStorageID(table_to_join.database_and_table_name); - StoragePtr table = DatabaseCatalog::instance().tryGetTable(joined_table_id); + StoragePtr table = DatabaseCatalog::instance().tryGetTable(joined_table_id, context); if (table) { if (dynamic_cast(table.get()) || diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 3c0d2159ef9..649cfa28e6e 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -431,7 +431,7 @@ void SystemLog::prepareTable() { String description = table_id.getNameForLogs(); - table = DatabaseCatalog::instance().tryGetTable(table_id); + table = DatabaseCatalog::instance().tryGetTable(table_id, context); if (table) { @@ -442,7 +442,7 @@ void SystemLog::prepareTable() { /// Rename the existing table. int suffix = 0; - while (DatabaseCatalog::instance().isTableExist({table_id.database_name, table_id.table_name + "_" + toString(suffix)})) + while (DatabaseCatalog::instance().isTableExist({table_id.database_name, table_id.table_name + "_" + toString(suffix)}, context)) ++suffix; auto rename = std::make_shared(); @@ -483,7 +483,7 @@ void SystemLog::prepareTable() interpreter.setInternal(true); interpreter.execute(); - table = DatabaseCatalog::instance().getTable(table_id); + table = DatabaseCatalog::instance().getTable(table_id, context); } is_prepared = true; diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index b5444f73b35..8467a98685d 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -96,7 +96,7 @@ static NamesAndTypesList getColumnsFromTableExpression(const ASTTableExpression else if (table_expression.database_and_table_name) { auto table_id = context.resolveStorageID(table_expression.database_and_table_name); - const auto & table = DatabaseCatalog::instance().getTable(table_id); + const auto & table = DatabaseCatalog::instance().getTable(table_id, context); const auto & columns = table->getColumns(); names_and_type_list = columns.getOrdinary(); materialized = columns.getMaterialized(); diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index e108db1af30..c94759897f5 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -96,7 +96,7 @@ std::shared_ptr interpretSubquery( else { auto table_id = context.resolveStorageID(table_expression); - const auto & storage = DatabaseCatalog::instance().getTable(table_id); + const auto & storage = DatabaseCatalog::instance().getTable(table_id, context); columns = storage->getColumns().getOrdinary(); select_query->replaceDatabaseAndTable(table_id); } diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index dabb0c12208..51b3d7eaef5 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -253,7 +253,7 @@ void MySQLHandler::comFieldList(ReadBuffer & payload) ComFieldList packet; packet.readPayload(payload); String database = connection_context.getCurrentDatabase(); - StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table}); + StoragePtr table_ptr = DatabaseCatalog::instance().getTable({database, packet.table}, connection_context); for (const NameAndTypePair & column: table_ptr->getColumns().getAll()) { ColumnDefinition column_definition( diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b8f3660179d..561634e11d7 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -474,7 +474,7 @@ void TCPHandler::processInsertQuery(const Settings & connection_settings) if (query_context->getSettingsRef().input_format_defaults_for_omitted_fields) { if (!table_id.empty()) - sendTableColumns(DatabaseCatalog::instance().getTable(table_id)->getColumns()); + sendTableColumns(DatabaseCatalog::instance().getTable(table_id, *query_context)->getColumns()); } } @@ -627,7 +627,7 @@ void TCPHandler::processTablesStatusRequest() for (const QualifiedTableName & table_name: request.tables) { auto resolved_id = connection_context.tryResolveStorageID({table_name.database, table_name.table}); - StoragePtr table = DatabaseCatalog::instance().tryGetTable(resolved_id); + StoragePtr table = DatabaseCatalog::instance().tryGetTable(resolved_id, connection_context); if (!table) continue; @@ -944,7 +944,7 @@ bool TCPHandler::receiveData(bool scalar) StoragePtr storage; /// If such a table does not exist, create it. if (resolved) - storage = DatabaseCatalog::instance().getTable(resolved); + storage = DatabaseCatalog::instance().getTable(resolved, *query_context); else { NamesAndTypesList columns = block.getNamesAndTypesList(); diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 7731cf3c06a..259d3d68eb0 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -361,7 +361,7 @@ bool StorageKafka::checkDependencies(const StorageID & table_id) // Check the dependencies are ready? for (const auto & db_tab : dependencies) { - auto table = DatabaseCatalog::instance().tryGetTable(db_tab); + auto table = DatabaseCatalog::instance().tryGetTable(db_tab, global_context); if (!table) return false; @@ -429,7 +429,7 @@ void StorageKafka::threadFunc() bool StorageKafka::streamToViews() { auto table_id = getStorageID(); - auto table = DatabaseCatalog::instance().getTable(table_id); + auto table = DatabaseCatalog::instance().getTable(table_id, global_context); if (!table) throw Exception("Engine table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index cd660407c89..633669cbf1b 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -401,7 +401,7 @@ void StorageLiveView::noUsersThread(std::shared_ptr storage, co if (drop_table) { - if (DatabaseCatalog::instance().tryGetTable(table_id)) + if (DatabaseCatalog::instance().tryGetTable(table_id, storage->global_context)) { try { diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 801b0b42ec4..fe62de224da 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -53,7 +53,7 @@ public: { return getStorageID().table_name + "_blocks"; } - StoragePtr getParentStorage() const { return DatabaseCatalog::instance().getTable(select_table_id); } + StoragePtr getParentStorage() const { return DatabaseCatalog::instance().getTable(select_table_id, global_context); } ASTPtr getInnerQuery() const { return inner_query->clone(); } ASTPtr getInnerSubQuery() const diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 95be1275d3d..47beeec196f 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -129,7 +129,7 @@ QueryProcessingStage::Enum StorageBuffer::getQueryProcessingStage(const Context { if (destination_id) { - auto destination = DatabaseCatalog::instance().getTable(destination_id); + auto destination = DatabaseCatalog::instance().getTable(destination_id, context); if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); @@ -153,7 +153,7 @@ Pipes StorageBuffer::read( if (destination_id) { - auto destination = DatabaseCatalog::instance().getTable(destination_id); + auto destination = DatabaseCatalog::instance().getTable(destination_id, context); if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); @@ -334,7 +334,7 @@ public: StoragePtr destination; if (storage.destination_id) { - destination = DatabaseCatalog::instance().tryGetTable(storage.destination_id); + destination = DatabaseCatalog::instance().tryGetTable(storage.destination_id, storage.global_context); if (destination.get() == &storage) throw Exception("Destination table is myself. Write will cause infinite loop.", ErrorCodes::INFINITE_LOOP); } @@ -434,7 +434,7 @@ bool StorageBuffer::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, con if (!destination_id) return false; - auto destination = DatabaseCatalog::instance().getTable(destination_id); + auto destination = DatabaseCatalog::instance().getTable(destination_id, query_context); if (destination.get() == this) throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP); @@ -602,7 +602,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds, bool loc */ try { - writeBlockToDestination(block_to_write, DatabaseCatalog::instance().tryGetTable(destination_id)); + writeBlockToDestination(block_to_write, DatabaseCatalog::instance().tryGetTable(destination_id, global_context)); } catch (...) { @@ -739,7 +739,7 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const S std::optional StorageBuffer::totalRows() const { std::optional underlying_rows; - auto underlying = DatabaseCatalog::instance().tryGetTable(destination_id); + auto underlying = DatabaseCatalog::instance().tryGetTable(destination_id, global_context); if (underlying) underlying_rows = underlying->totalRows(); diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 6d6c1f66569..10a4482c801 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -75,7 +75,7 @@ public: { if (!destination_id) return false; - auto dest = DatabaseCatalog::instance().tryGetTable(destination_id); + auto dest = DatabaseCatalog::instance().tryGetTable(destination_id, global_context); if (dest && dest.get() != this) return dest->supportsPrewhere(); return false; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index a565c8c6260..2afa7f7d713 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -149,7 +149,7 @@ StorageMaterializedView::StorageMaterializedView( create_interpreter.setInternal(true); create_interpreter.execute(); - target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->database, manual_create_query->table})->getStorageID(); + target_table_id = DatabaseCatalog::instance().getTable({manual_create_query->database, manual_create_query->table}, global_context)->getStorageID(); } if (!select_table_id.empty()) @@ -204,7 +204,7 @@ BlockOutputStreamPtr StorageMaterializedView::write(const ASTPtr & query, const static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, const StorageID & target_table_id) { - if (DatabaseCatalog::instance().tryGetTable(target_table_id)) + if (DatabaseCatalog::instance().tryGetTable(target_table_id, global_context)) { /// We create and execute `drop` query for internal table. auto drop_query = std::make_shared(); @@ -362,12 +362,12 @@ void StorageMaterializedView::shutdown() StoragePtr StorageMaterializedView::getTargetTable() const { - return DatabaseCatalog::instance().getTable(target_table_id); + return DatabaseCatalog::instance().getTable(target_table_id, global_context); } StoragePtr StorageMaterializedView::tryGetTargetTable() const { - return DatabaseCatalog::instance().tryGetTable(target_table_id); + return DatabaseCatalog::instance().tryGetTable(target_table_id, global_context); } Strings StorageMaterializedView::getDataPaths() const diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index f6114d709b6..c6af41cc163 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -990,7 +990,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma case PartitionCommand::MoveDestinationType::TABLE: checkPartitionCanBeDropped(command.partition); String dest_database = context.resolveDatabase(command.to_database); - auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}); + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, context); movePartitionToTable(dest_storage, command.partition, context); break; } @@ -1002,7 +1002,7 @@ void StorageMergeTree::alterPartition(const ASTPtr & query, const PartitionComma { checkPartitionCanBeDropped(command.partition); String from_database = context.resolveDatabase(command.from_database); - auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}); + auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}, context); replacePartitionFrom(from_storage, command.partition, command.replace, context); } break; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b4ca97602c4..00565e777ae 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1570,7 +1570,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) auto clone_data_parts_from_source_table = [&] () -> size_t { - source_table = DatabaseCatalog::instance().tryGetTable(source_table_id); + source_table = DatabaseCatalog::instance().tryGetTable(source_table_id, global_context); if (!source_table) { LOG_DEBUG(log, "Can't use {} as source table for REPLACE PARTITION command. It does not exist.", source_table_id.getNameForLogs()); @@ -3485,7 +3485,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part case PartitionCommand::MoveDestinationType::TABLE: checkPartitionCanBeDropped(command.partition); String dest_database = query_context.resolveDatabase(command.to_database); - auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}); + auto dest_storage = DatabaseCatalog::instance().getTable({dest_database, command.to_table}, query_context); movePartitionToTable(dest_storage, command.partition, query_context); break; } @@ -3496,7 +3496,7 @@ void StorageReplicatedMergeTree::alterPartition(const ASTPtr & query, const Part { checkPartitionCanBeDropped(command.partition); String from_database = query_context.resolveDatabase(command.from_database); - auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}); + auto from_storage = DatabaseCatalog::instance().getTable({from_database, command.from_table}, query_context); replacePartitionFrom(from_storage, command.partition, command.replace, query_context); } break; diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 711612cf9e5..19d1172f1ff 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -84,7 +84,7 @@ ColumnsDescription getStructureOfRemoteTableInShard( else { if (shard_info.isLocal()) - return DatabaseCatalog::instance().getTable(table_id)->getColumns(); + return DatabaseCatalog::instance().getTable(table_id, context)->getColumns(); /// Request for a table description query = "DESC TABLE " + table_id.getFullTableName(); From 23d7947549cb44b45bdcf053af29aa45e9a2a211 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 05:08:48 +0300 Subject: [PATCH 060/150] Fix constraints for DEFAULT fields; fix constraints for temporary tables; improve performance of INSERT for small blocks --- src/Core/ExternalTable.cpp | 2 +- src/Interpreters/DatabaseCatalog.cpp | 8 +++-- src/Interpreters/DatabaseCatalog.h | 7 ++++- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 24 +++++++++------ src/Server/TCPHandler.cpp | 2 +- .../01286_constraints_on_default.reference | 2 ++ .../01286_constraints_on_default.sql | 29 +++++++++++++++++++ 9 files changed, 62 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/01286_constraints_on_default.reference create mode 100644 tests/queries/0_stateless/01286_constraints_on_default.sql diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 62a99cea97e..5ec6980dbfa 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -164,7 +164,7 @@ void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, /// Create table NamesAndTypesList columns = sample_block.getNamesAndTypesList(); - auto temporary_table = TemporaryTableHolder(context, ColumnsDescription{columns}); + auto temporary_table = TemporaryTableHolder(context, ColumnsDescription{columns}, {}); auto storage = temporary_table.getTable(); context.addExternalTable(data->table_name, std::move(temporary_table)); BlockOutputStreamPtr output = storage->write(ASTPtr(), context); diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 7606fdb255a..c55e60898b9 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -58,13 +58,17 @@ TemporaryTableHolder::TemporaryTableHolder(const Context & context_, } -TemporaryTableHolder::TemporaryTableHolder(const Context & context_, const ColumnsDescription & columns, const ASTPtr & query) +TemporaryTableHolder::TemporaryTableHolder( + const Context & context_, + const ColumnsDescription & columns, + const ConstraintsDescription & constraints, + const ASTPtr & query) : TemporaryTableHolder ( context_, [&](const StorageID & table_id) { - return StorageMemory::create(table_id, ColumnsDescription{columns}, ConstraintsDescription{}); + return StorageMemory::create(table_id, ColumnsDescription{columns}, ConstraintsDescription{constraints}); }, query ) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index a481e3d7e5e..a274d294af0 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -21,6 +21,7 @@ class Context; class IDatabase; class Exception; class ColumnsDescription; +struct ConstraintsDescription; using DatabasePtr = std::shared_ptr; using DatabaseAndTable = std::pair; @@ -71,7 +72,11 @@ struct TemporaryTableHolder : boost::noncopyable TemporaryTableHolder(const Context & context, const Creator & creator, const ASTPtr & query = {}); /// Creates temporary table with Engine=Memory - TemporaryTableHolder(const Context & context, const ColumnsDescription & columns, const ASTPtr & query = {}); + TemporaryTableHolder( + const Context & context, + const ColumnsDescription & columns, + const ConstraintsDescription & constraints, + const ASTPtr & query = {}); TemporaryTableHolder(TemporaryTableHolder && rhs); TemporaryTableHolder & operator = (TemporaryTableHolder && rhs); diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 37a358c3d28..9e616b04dab 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -103,7 +103,7 @@ public: Block sample = interpreter->getSampleBlock(); NamesAndTypesList columns = sample.getNamesAndTypesList(); - auto external_storage_holder = std::make_shared(context, ColumnsDescription{columns}); + auto external_storage_holder = std::make_shared(context, ColumnsDescription{columns}, ConstraintsDescription{}); StoragePtr external_storage = external_storage_holder->getTable(); /** We replace the subquery with the name of the temporary table. diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 147ef7d739b..6d35e25ba44 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -637,7 +637,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, if (create.if_not_exists && context.tryResolveStorageID({"", table_name}, Context::ResolveExternal)) return false; - auto temporary_table = TemporaryTableHolder(context, properties.columns, query_ptr); + auto temporary_table = TemporaryTableHolder(context, properties.columns, properties.constraints, query_ptr); context.getSessionContext().addExternalTable(table_name, std::move(temporary_table)); return true; } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b6efa5d6d46..149a78c3803 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -233,6 +233,21 @@ BlockIO InterpreterInsertQuery::execute() else out = std::make_shared(table, context, query_ptr, no_destination); + /// Note that we wrap transforms one on top of another, so we write them in reverse of data processing order. + + /// Checking constraints. It must be done after calculation of all defaults, so we can check them on calculated columns. + if (const auto & constraints = table->getConstraints(); !constraints.empty()) + out = std::make_shared( + query.table_id, out, out->getHeader(), table->getConstraints(), context); + + /// Actually we don't know structure of input blocks from query/table, + /// because some clients break insertion protocol (columns != header) + out = std::make_shared( + out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context); + + /// It's important to squash blocks as early as possible (before other transforms), + /// because other transforms may work inefficient if block size is small. + /// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side. /// Client-side bufferization might cause excessive timeouts (especially in case of big blocks). if (!(context.getSettingsRef().insert_distributed_sync && table->isRemote()) && !no_squash && !query.watch) @@ -244,15 +259,6 @@ BlockIO InterpreterInsertQuery::execute() context.getSettingsRef().min_insert_block_size_bytes); } - /// Actually we don't know structure of input blocks from query/table, - /// because some clients break insertion protocol (columns != header) - out = std::make_shared( - out, query_sample_block, out->getHeader(), table->getColumns().getDefaults(), context); - - if (const auto & constraints = table->getConstraints(); !constraints.empty()) - out = std::make_shared( - query.table_id, out, query_sample_block, table->getConstraints(), context); - auto out_wrapper = std::make_shared(out); out_wrapper->setProcessListElement(context.getProcessListElement()); out = std::move(out_wrapper); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b8f3660179d..65ed1f5eb90 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -948,7 +948,7 @@ bool TCPHandler::receiveData(bool scalar) else { NamesAndTypesList columns = block.getNamesAndTypesList(); - auto temporary_table = TemporaryTableHolder(*query_context, ColumnsDescription{columns}); + auto temporary_table = TemporaryTableHolder(*query_context, ColumnsDescription{columns}, {}); storage = temporary_table.getTable(); query_context->addExternalTable(temporary_id.table_name, std::move(temporary_table)); } diff --git a/tests/queries/0_stateless/01286_constraints_on_default.reference b/tests/queries/0_stateless/01286_constraints_on_default.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/01286_constraints_on_default.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/01286_constraints_on_default.sql b/tests/queries/0_stateless/01286_constraints_on_default.sql new file mode 100644 index 00000000000..d150bac15b5 --- /dev/null +++ b/tests/queries/0_stateless/01286_constraints_on_default.sql @@ -0,0 +1,29 @@ +DROP TABLE IF EXISTS default_constraints; +CREATE TABLE default_constraints +( + x UInt8, + y UInt8 DEFAULT x + 1, + CONSTRAINT c CHECK y < 5 +) ENGINE = Memory; + +INSERT INTO default_constraints (x) SELECT number FROM system.numbers LIMIT 5; -- { serverError 469 } +INSERT INTO default_constraints (x) VALUES (0),(1),(2),(3),(4); -- { serverError 469 } + +SELECT y, throwIf(NOT y < 5) FROM default_constraints; +SELECT count() FROM default_constraints; + +DROP TABLE default_constraints; + + +CREATE TEMPORARY TABLE default_constraints +( + x UInt8, + y UInt8 DEFAULT x + 1, + CONSTRAINT c CHECK y < 5 +); + +INSERT INTO default_constraints (x) SELECT number FROM system.numbers LIMIT 5; -- { serverError 469 } +INSERT INTO default_constraints (x) VALUES (0),(1),(2),(3),(4); -- { serverError 469 } + +SELECT y, throwIf(NOT y < 5) FROM default_constraints; +SELECT count() FROM default_constraints; From b7b63801411c64bdeffc7439cb6aeaea509ddc4f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 05:30:49 +0300 Subject: [PATCH 061/150] Add performance test --- .../insert_select_default_small_block.xml | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) create mode 100644 tests/performance/insert_select_default_small_block.xml diff --git a/tests/performance/insert_select_default_small_block.xml b/tests/performance/insert_select_default_small_block.xml new file mode 100644 index 00000000000..12e67b09d2f --- /dev/null +++ b/tests/performance/insert_select_default_small_block.xml @@ -0,0 +1,38 @@ + + 1 + + +CREATE TABLE insert_small_block_performance +( + `x` String, + `a` DEFAULT SHA256(x), + `b` DEFAULT SHA256(toString(a)), + `c` DEFAULT SHA256(toString(b)), + `d` DEFAULT SHA256(toString(c)), + `e` DEFAULT SHA256(toString(d)), + `f` DEFAULT SHA256(toString(e)), + `g` DEFAULT SHA256(toString(f)), + `h` DEFAULT SHA256(toString(g)), + `i` DEFAULT SHA256(toString(h)), + `j` DEFAULT SHA256(toString(i)), + `k` DEFAULT SHA256(toString(j)), + `l` DEFAULT SHA256(toString(k)), + `m` DEFAULT SHA256(toString(l)), + `n` DEFAULT SHA256(toString(m)), + `o` DEFAULT SHA256(toString(n)), + `p` DEFAULT SHA256(toString(o)), + `q` DEFAULT SHA256(toString(p)), + `r` DEFAULT SHA256(toString(q)), + `s` DEFAULT SHA256(toString(r)), + `t` DEFAULT SHA256(toString(s)), + `u` DEFAULT SHA256(toString(t)), + `v` DEFAULT SHA256(toString(u)), + `w` DEFAULT SHA256(toString(v)) +) +ENGINE = Null; + + + INSERT INTO insert_small_block_performance (x) SELECT toString(number) FROM numbers(10000); + + DROP TABLE IF EXISTS insert_small_block_performance + From 9120495c4861aaf71a693ec0ad06c0333e68c4d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 05:31:45 +0300 Subject: [PATCH 062/150] Increase chunk size for parallel parsing just in case --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 142e0872d72..8cf90fa146a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -126,7 +126,7 @@ struct Settings : public SettingsCollection M(SettingBool, force_optimize_skip_unused_shards_no_nested, false, "Do not apply force_optimize_skip_unused_shards for nested Distributed tables.", 0) \ \ M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \ - M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \ + M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (10 * 1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \ \ M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \ M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \ From 0e27f476323ca8c17ae8ea0b87159128825f0c2d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 May 2020 11:42:55 +0300 Subject: [PATCH 063/150] Try update test. --- ...in_insert_block_size_rows_for_materialized_views.reference | 4 ++-- ...01278_min_insert_block_size_rows_for_materialized_views.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.reference b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.reference index ed22b7e1e35..e4872ddeddd 100644 --- a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.reference +++ b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.reference @@ -1,4 +1,4 @@ 0 0 -100000 -200000 +800000 +1600000 diff --git a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh index 7e08c930f67..c9a4de647b9 100755 --- a/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh +++ b/tests/queries/0_stateless/01278_min_insert_block_size_rows_for_materialized_views.sh @@ -54,7 +54,7 @@ EOL echo "create table null_01278 as data_01278 Engine=Null();" | execute for i in $(seq 1 $TEST_01278_PARTS); do echo "create table part_01278_$i as data_01278 Engine=Buffer(currentDatabase(), null_01278, 1, 86400, 86400, 1e5, 1e6, 10e6, 100e6);" - echo "create materialized view mv_01278_$i to part_01278_$i as select * from data_01278 where key%$TEST_01278_PARTS+1 == $i;" + echo "create materialized view mv_01278_$i to part_01278_$i as select * from data_01278 where key%$TEST_01278_PARTS+1 != $i;" done | execute echo "create table out_01278 as data_01278 Engine=Merge(currentDatabase(), 'part_01278_');" | execute From b885337454edf2f690535c3e76f455a357c0bc89 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 29 May 2020 10:44:10 +0200 Subject: [PATCH 064/150] Virtual columns for accessing kafka message headers --- src/Storages/Kafka/KafkaBlockInputStream.cpp | 20 ++++++++++++-- .../Kafka/ReadBufferFromKafkaConsumer.h | 1 + src/Storages/Kafka/StorageKafka.cpp | 5 +++- tests/integration/test_storage_kafka/test.py | 26 +++++++++---------- 4 files changed, 36 insertions(+), 16 deletions(-) diff --git a/src/Storages/Kafka/KafkaBlockInputStream.cpp b/src/Storages/Kafka/KafkaBlockInputStream.cpp index 6ae7e2606b6..3e4533f8bb2 100644 --- a/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -20,8 +20,7 @@ KafkaBlockInputStream::KafkaBlockInputStream( , max_block_size(max_block_size_) , commit_in_suffix(commit_in_suffix_) , non_virtual_header(storage.getSampleBlockNonMaterialized()) - , virtual_header(storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp","_timestamp_ms"})) - + , virtual_header(storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp","_timestamp_ms","_headers.name","_headers.value"})) { context.setSetting("input_format_skip_unknown_fields", 1u); // Always skip unknown fields regardless of the context (JSON or TSKV) context.setSetting("input_format_allow_errors_ratio", 0.); @@ -141,6 +140,21 @@ Block KafkaBlockInputStream::readImpl() auto offset = buffer->currentOffset(); auto partition = buffer->currentPartition(); auto timestamp_raw = buffer->currentTimestamp(); + auto header_list = buffer->currentHeaderList(); + + Array headers_names; + Array headers_values; + + if (!header_list.empty()) + { + headers_names.reserve(header_list.size()); + headers_values.reserve(header_list.size()); + for (const auto & header : header_list) + { + headers_names.emplace_back(header.get_name()); + headers_values.emplace_back(static_cast(header.get_value())); + } + } for (size_t i = 0; i < new_rows; ++i) { @@ -159,6 +173,8 @@ Block KafkaBlockInputStream::readImpl() virtual_columns[4]->insertDefault(); virtual_columns[5]->insertDefault(); } + virtual_columns[6]->insert(headers_names); + virtual_columns[7]->insert(headers_values); } total_rows = total_rows + new_rows; diff --git a/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h b/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h index e90e3b48881..7449f58c838 100644 --- a/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h +++ b/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h @@ -49,6 +49,7 @@ public: auto currentOffset() const { return current[-1].get_offset(); } auto currentPartition() const { return current[-1].get_partition(); } auto currentTimestamp() const { return current[-1].get_timestamp(); } + const auto & currentHeaderList() const { return current[-1].get_header_list(); } private: using Messages = std::vector; diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index d1f350b02a6..d1014fdb0f8 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -726,7 +727,9 @@ NamesAndTypesList StorageKafka::getVirtuals() const {"_offset", std::make_shared()}, {"_partition", std::make_shared()}, {"_timestamp", std::make_shared(std::make_shared())}, - {"_timestamp_ms", std::make_shared(std::make_shared(3))} + {"_timestamp_ms", std::make_shared(std::make_shared(3))}, + {"_headers.name", std::make_shared(std::make_shared())}, + {"_headers.value", std::make_shared(std::make_shared())} }; } diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index d89684e2131..82b409aa85e 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -840,18 +840,18 @@ def test_kafka_virtual_columns2(kafka_cluster): kafka_format = 'JSONEachRow'; CREATE MATERIALIZED VIEW test.view Engine=Log AS - SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms) FROM test.kafka; + SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; ''') producer = KafkaProducer(bootstrap_servers="localhost:9092") - producer.send(topic='virt2_0', value=json.dumps({'value': 1}), partition=0, key='k1', timestamp_ms=1577836801001) - producer.send(topic='virt2_0', value=json.dumps({'value': 2}), partition=0, key='k2', timestamp_ms=1577836802002) + producer.send(topic='virt2_0', value=json.dumps({'value': 1}), partition=0, key='k1', timestamp_ms=1577836801001, headers=[('content-encoding', b'base64')]) + producer.send(topic='virt2_0', value=json.dumps({'value': 2}), partition=0, key='k2', timestamp_ms=1577836802002, headers=[('empty_value', ''),('', 'empty name'), ('',''), ('repetition', '1'), ('repetition', '2')]) producer.flush() time.sleep(1) - producer.send(topic='virt2_0', value=json.dumps({'value': 3}), partition=1, key='k3', timestamp_ms=1577836803003) - producer.send(topic='virt2_0', value=json.dumps({'value': 4}), partition=1, key='k4', timestamp_ms=1577836804004) + producer.send(topic='virt2_0', value=json.dumps({'value': 3}), partition=1, key='k3', timestamp_ms=1577836803003, headers=[('b', 'b'),('a', 'a')]) + producer.send(topic='virt2_0', value=json.dumps({'value': 4}), partition=1, key='k4', timestamp_ms=1577836804004, headers=[('a', 'a'),('b', 'b')]) producer.flush() time.sleep(1) @@ -869,14 +869,14 @@ def test_kafka_virtual_columns2(kafka_cluster): result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) expected = '''\ -1 k1 virt2_0 0 0 1577836801 1577836801001 -2 k2 virt2_0 0 1 1577836802 1577836802002 -3 k3 virt2_0 1 0 1577836803 1577836803003 -4 k4 virt2_0 1 1 1577836804 1577836804004 -5 k5 virt2_1 0 0 1577836805 1577836805005 -6 k6 virt2_1 0 1 1577836806 1577836806006 -7 k7 virt2_1 1 0 1577836807 1577836807007 -8 k8 virt2_1 1 1 1577836808 1577836808008 +1 k1 virt2_0 0 0 1577836801 1577836801001 ['content-encoding'] ['base64'] +2 k2 virt2_0 0 1 1577836802 1577836802002 ['empty_value','','','repetition','repetition'] ['','empty name','','1','2'] +3 k3 virt2_0 1 0 1577836803 1577836803003 ['b','a'] ['b','a'] +4 k4 virt2_0 1 1 1577836804 1577836804004 ['a','b'] ['a','b'] +5 k5 virt2_1 0 0 1577836805 1577836805005 [] [] +6 k6 virt2_1 0 1 1577836806 1577836806006 [] [] +7 k7 virt2_1 1 0 1577836807 1577836807007 [] [] +8 k8 virt2_1 1 1 1577836808 1577836808008 [] [] ''' assert TSV(result) == TSV(expected) From 998a9f4c0b7bb099e55f25a96bd79c1a42fb9db7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 May 2020 12:52:30 +0300 Subject: [PATCH 065/150] Try fix copier tests. --- programs/copier/ClusterCopier.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index d50b89738aa..d6a2fe103d1 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1690,7 +1690,7 @@ std::set ClusterCopier::getShardPartitions(const ConnectionTimeouts & ti Context local_context = context; local_context.setSettings(task_cluster->settings_pull); - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().in); + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()); std::set res; if (block) From bd03d1acb964978638d57e98d8d675b857ddc289 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 May 2020 13:08:11 +0300 Subject: [PATCH 066/150] Remove hardcoded database from some tests and add retries to clickhouse-test --- tests/clickhouse-test | 19 +- .../00365_statistics_in_formats.sh | 20 +- ...00563_insert_into_remote_and_zookeeper.sql | 16 +- ..._replace_partition_from_table_zookeeper.sh | 181 +++++++++--------- .../queries/0_stateless/00763_lock_buffer.sh | 14 +- ...ong_lock_buffer_alter_destination_table.sh | 18 +- ...ted_minimalistic_part_header_zookeeper.sql | 48 ++--- ...ndex_granularity_replicated_merge_tree.sql | 164 ++++++++-------- .../0_stateless/00937_test_use_header_csv.sh | 10 +- .../queries/0_stateless/00942_mutate_index.sh | 20 +- .../0_stateless/00943_materialize_index.sh | 46 ++--- .../00944_clear_index_in_partition.sh | 26 +-- ...5_indices_mutation_replicated_zookeeper.sh | 30 +-- .../01019_parallel_parsing_cancel.sh | 16 +- ...1122_totals_rollup_having_block_header.sql | 16 +- .../0_stateless/01124_view_bad_types.sql | 16 +- ...285_date_datetime_key_condition.reference} | 0 .../01285_date_datetime_key_condition.sql | 22 +++ .../00155_date_datetime_key_condition.sql | 22 --- 19 files changed, 360 insertions(+), 344 deletions(-) rename tests/queries/{1_stateful/00155_date_datetime_key_condition.reference => 0_stateless/01285_date_datetime_key_condition.reference} (100%) create mode 100644 tests/queries/0_stateless/01285_date_datetime_key_condition.sql delete mode 100644 tests/queries/1_stateful/00155_date_datetime_key_condition.sql diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 1a2735296e8..1b45e4bb5d3 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -331,6 +331,19 @@ def run_tests_array(all_tests_with_params): server_logs_level = "warning" +def check_server_started(client, retry_count): + while retry_count > 0: + clickhouse_proc = Popen(shlex.split(client), stdin=PIPE, stdout=PIPE, stderr=PIPE) + (stdout, stderr) = clickhouse_proc.communicate("SELECT 1") + if clickhouse_proc.returncode != 0 or not stdout.startswith("1"): + retry_count -= 1 + sleep(0.5) + else: + return True + + return False + + def main(args): global SERVER_DIED global exit_code @@ -344,6 +357,9 @@ def main(args): return stdout.startswith('1') + if not check_server_started(args.client, args.server_check_retries): + raise Exception("clickhouse-server is not responding. Cannot execute 'SELECT 1' query.") + base_dir = os.path.abspath(args.queries) tmp_dir = os.path.abspath(args.tmp) @@ -360,7 +376,7 @@ def main(args): os.environ.setdefault("CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL", server_logs_level) if args.zookeeper is None: - code, out = commands.getstatusoutput(args.extract_from_config +" --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') + code, out = commands.getstatusoutput(args.extract_from_config + " --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l') try: if int(out) > 0: args.zookeeper = True @@ -570,6 +586,7 @@ if __name__ == '__main__': parser.add_argument('--parallel', default='1/1', help='One parallel test run number/total') parser.add_argument('-j', '--jobs', default=1, nargs='?', type=int, help='Run all tests in parallel') parser.add_argument('-U', '--unified', default=3, type=int, help='output NUM lines of unified context') + parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started') parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests') parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests') diff --git a/tests/queries/0_stateless/00365_statistics_in_formats.sh b/tests/queries/0_stateless/00365_statistics_in_formats.sh index f0e23337806..fb79b23f78e 100755 --- a/tests/queries/0_stateless/00365_statistics_in_formats.sh +++ b/tests/queries/0_stateless/00365_statistics_in_formats.sh @@ -3,16 +3,16 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.numbers"; -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.numbers (number UInt64) engine = MergeTree order by number"; -$CLICKHOUSE_CLIENT --query="INSERT INTO test.numbers select * from system.numbers limit 10"; +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS numbers"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE numbers (number UInt64) engine = MergeTree order by number"; +$CLICKHOUSE_CLIENT --query="INSERT INTO numbers select * from system.numbers limit 10"; -$CLICKHOUSE_CLIENT --query="SELECT number FROM test.numbers LIMIT 10 FORMAT JSON" | grep 'rows_read'; -$CLICKHOUSE_CLIENT --query="SELECT number FROM test.numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read'; -$CLICKHOUSE_CLIENT --query="SELECT number FROM test.numbers LIMIT 10 FORMAT XML" | grep 'rows_read'; +$CLICKHOUSE_CLIENT --query="SELECT number FROM numbers LIMIT 10 FORMAT JSON" | grep 'rows_read'; +$CLICKHOUSE_CLIENT --query="SELECT number FROM numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read'; +$CLICKHOUSE_CLIENT --query="SELECT number FROM numbers LIMIT 10 FORMAT XML" | grep 'rows_read'; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM test.numbers LIMIT 10 FORMAT JSON" | grep 'rows_read'; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM test.numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read'; -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM test.numbers LIMIT 10 FORMAT XML" | grep 'rows_read'; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM numbers LIMIT 10 FORMAT JSON" | grep 'rows_read'; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM numbers LIMIT 10 FORMAT JSONCompact" | grep 'rows_read'; +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d "SELECT number FROM numbers LIMIT 10 FORMAT XML" | grep 'rows_read'; -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.numbers"; \ No newline at end of file +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS numbers"; diff --git a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql index d26e947b54f..b4579e6aac3 100644 --- a/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql +++ b/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql @@ -1,14 +1,14 @@ -- Check that settings are correctly passed through Distributed table -DROP TABLE IF EXISTS test.simple; -CREATE TABLE test.simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/test/simple', '1') ORDER BY d; +DROP TABLE IF EXISTS simple; +CREATE TABLE simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/test/simple', '1') ORDER BY d; -- TODO: replace '127.0.0.2' -> '127.0.0.1' after a fix -INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (1); -INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (1); +INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (1); +INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (1); SET insert_deduplicate=0; -INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (2); -INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (2); +INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (2); +INSERT INTO TABLE FUNCTION remote('127.0.0.2', currentDatabase(), 'simple') VALUES (2); -SELECT * FROM remote('127.0.0.2', 'test', 'simple') ORDER BY d; -DROP TABLE test.simple; \ No newline at end of file +SELECT * FROM remote('127.0.0.2', currentDatabase(), 'simple') ORDER BY d; +DROP TABLE simple; diff --git a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh index 44ab712bcf8..2f3514ae2f5 100755 --- a/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/00626_replace_partition_from_table_zookeeper.sh @@ -26,144 +26,143 @@ function query_with_retry echo "Query '$1' failed with '$result'" } -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r1;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r2;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r2;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.dst_r1 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '1') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.dst_r2 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '2') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE dst_r1 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '1') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE dst_r2 (p UInt64, k String, d UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/test/dst_1', '2') PARTITION BY p ORDER BY k SETTINGS old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=0;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (0, '0', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (2, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (0, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (2, '0', 1);" $CLICKHOUSE_CLIENT --query="SELECT 'Initial';" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (0, '1', 2);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2), (1, '2', 2);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (2, '1', 2);" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (0, '1', 2);" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2), (1, '2', 2);" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (2, '1', 2);" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.src;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM src;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE simple';" -query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;" -query_with_retry "ALTER TABLE test.src DROP PARTITION 1;" +query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;" +query_with_retry "ALTER TABLE src DROP PARTITION 1;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.src;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM src;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE empty';" -query_with_retry "ALTER TABLE test.src DROP PARTITION 1;" -query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;" +query_with_retry "ALTER TABLE src DROP PARTITION 1;" +query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE recursive';" -query_with_retry "ALTER TABLE test.dst_r1 DROP PARTITION 1;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2), (1, '2', 2);" +query_with_retry "ALTER TABLE dst_r1 DROP PARTITION 1;" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2), (1, '2', 2);" $CLICKHOUSE_CLIENT --query="CREATE table test_block_numbers (m UInt64) ENGINE MergeTree() ORDER BY tuple();" -$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database='test' AND table='dst_r1' AND active AND name LIKE '1_%';" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database=currentDatabase() AND table='dst_r1' AND active AND name LIKE '1_%';" -query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database='test' AND table='dst_r1' AND active AND name LIKE '1_%';" +$CLICKHOUSE_CLIENT --query="INSERT INTO test_block_numbers SELECT max(max_block_number) AS m FROM system.parts WHERE database=currentDatabase() AND table='dst_r1' AND active AND name LIKE '1_%';" $CLICKHOUSE_CLIENT --query="SELECT (max(m) - min(m) > 1) AS new_block_is_generated FROM test_block_numbers;" $CLICKHOUSE_CLIENT --query="DROP TABLE test_block_numbers;" $CLICKHOUSE_CLIENT --query="SELECT 'ATTACH FROM';" -query_with_retry "ALTER TABLE test.dst_r1 DROP PARTITION 1;" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;" +query_with_retry "ALTER TABLE dst_r1 DROP PARTITION 1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE src;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);" +$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r2 VALUES (1, '1', 2);" -query_with_retry "ALTER TABLE test.dst_r2 ATTACH PARTITION 1 FROM test.src;" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r2 VALUES (1, '1', 2);" +query_with_retry "ALTER TABLE dst_r2 ATTACH PARTITION 1 FROM src;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE with fetch';" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" +$CLICKHOUSE_CLIENT --query="DROP TABLE src;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" # Stop replication at the second replica and remove source table to use fetch instead of copying -$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES test.dst_r2;" -query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;" -$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES dst_r2;" +query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;" +$CLICKHOUSE_CLIENT --query="DROP TABLE src;" +$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES dst_r2;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'REPLACE with fetch of merged';" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.src;" -query_with_retry "ALTER TABLE test.dst_r1 DROP PARTITION 1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" +query_with_retry "ALTER TABLE dst_r1 DROP PARTITION 1;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '0', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.src VALUES (1, '1', 1);" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" +$CLICKHOUSE_CLIENT --query="CREATE TABLE src (p UInt64, k String, d UInt64) ENGINE = MergeTree PARTITION BY p ORDER BY k;" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '0', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO src VALUES (1, '1', 1);" +$CLICKHOUSE_CLIENT --query="INSERT INTO dst_r1 VALUES (1, '1', 2); -- trash part to be deleted" -$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES test.dst_r2;" -query_with_retry "ALTER TABLE test.dst_r1 REPLACE PARTITION 1 FROM test.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.src;" +$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM STOP REPLICATION QUEUES dst_r2;" +query_with_retry "ALTER TABLE dst_r1 REPLACE PARTITION 1 FROM src;" +$CLICKHOUSE_CLIENT --query="DROP TABLE src;" # do not wait other replicas to execute OPTIMIZE -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r1;" -query_with_retry "OPTIMIZE TABLE test.dst_r1 PARTITION 1;" "--replication_alter_partitions_sync=0 --optimize_throw_if_noop=1" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;" +query_with_retry "OPTIMIZE TABLE dst_r1 PARTITION 1;" "--replication_alter_partitions_sync=0 --optimize_throw_if_noop=1" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM test.dst_r1;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM dst_r1;" -$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM START REPLICATION QUEUES dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM START MERGES dst_r2;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d), uniqExact(_part) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'After restart';" -$CLICKHOUSE_CLIENT --query="USE test;" -$CLICKHOUSE_CLIENT --query="SYSTEM RESTART REPLICA test.dst_r1;" +$CLICKHOUSE_CLIENT --query="SYSTEM RESTART REPLICA dst_r1;" $CLICKHOUSE_CLIENT --query="SYSTEM RESTART REPLICAS;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" $CLICKHOUSE_CLIENT --query="SELECT 'DETACH+ATTACH PARTITION';" -query_with_retry "ALTER TABLE test.dst_r1 DETACH PARTITION 0;" -query_with_retry "ALTER TABLE test.dst_r1 DETACH PARTITION 1;" -query_with_retry "ALTER TABLE test.dst_r1 DETACH PARTITION 2;" -query_with_retry "ALTER TABLE test.dst_r1 ATTACH PARTITION 1;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r1;" -$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA test.dst_r2;" -$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM test.dst_r2;" +query_with_retry "ALTER TABLE dst_r1 DETACH PARTITION 0;" +query_with_retry "ALTER TABLE dst_r1 DETACH PARTITION 1;" +query_with_retry "ALTER TABLE dst_r1 DETACH PARTITION 2;" +query_with_retry "ALTER TABLE dst_r1 ATTACH PARTITION 1;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r1;" +$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA dst_r2;" +$CLICKHOUSE_CLIENT --query="SELECT count(), sum(d) FROM dst_r2;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.src;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r1;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.dst_r2;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS src;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS dst_r2;" diff --git a/tests/queries/0_stateless/00763_lock_buffer.sh b/tests/queries/0_stateless/00763_lock_buffer.sh index 3146ce96132..d320ae51a37 100755 --- a/tests/queries/0_stateless/00763_lock_buffer.sh +++ b/tests/queries/0_stateless/00763_lock_buffer.sh @@ -4,20 +4,20 @@ set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.mt_00763_2" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.buffer_00763_2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mt_00763_2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS buffer_00763_2" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.buffer_00763_2 (s String) ENGINE = Buffer(test, mt_00763_2, 1, 1, 1, 1, 1, 1, 1)" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_2 (s String) ENGINE = Buffer('$CLICKHOUSE_DATABASE', mt_00763_2, 1, 1, 1, 1, 1, 1, 1)" function thread1() { - seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS test.mt_00763_2; CREATE TABLE test.mt_00763_2 (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO test.mt_00763_2 SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: + seq 1 500 | sed -r -e 's/.+/DROP TABLE IF EXISTS mt_00763_2; CREATE TABLE mt_00763_2 (s String) ENGINE = MergeTree ORDER BY s; INSERT INTO mt_00763_2 SELECT toString(number) FROM numbers(10);/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: } function thread2() { - seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM test.buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' + seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' } thread1 & @@ -25,5 +25,5 @@ thread2 & wait -${CLICKHOUSE_CLIENT} --query="DROP TABLE test.mt_00763_2" -${CLICKHOUSE_CLIENT} --query="DROP TABLE test.buffer_00763_2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mt_00763_2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE buffer_00763_2" diff --git a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh index 059d70253b2..26bdb712ec5 100755 --- a/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh +++ b/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh @@ -6,21 +6,21 @@ CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.mt_00763_1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS test.buffer_00763_1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mt_00763_1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS buffer_00763_1" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.buffer_00763_1 (s String) ENGINE = Buffer(test, mt_00763_1, 1, 1, 1, 1, 1, 1, 1)" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE test.mt_00763_1 (x UInt32, s String) ENGINE = MergeTree ORDER BY x" -${CLICKHOUSE_CLIENT} --query="INSERT INTO test.mt_00763_1 VALUES (1, '1'), (2, '2'), (3, '3')" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE buffer_00763_1 (s String) ENGINE = Buffer($CLICKHOUSE_DATABASE, mt_00763_1, 1, 1, 1, 1, 1, 1, 1)" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE mt_00763_1 (x UInt32, s String) ENGINE = MergeTree ORDER BY x" +${CLICKHOUSE_CLIENT} --query="INSERT INTO mt_00763_1 VALUES (1, '1'), (2, '2'), (3, '3')" function thread1() { - seq 1 300 | sed -r -e 's/.+/ALTER TABLE test.mt_00763_1 MODIFY column s UInt32; ALTER TABLE test.mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: + seq 1 300 | sed -r -e 's/.+/ALTER TABLE mt_00763_1 MODIFY column s UInt32; ALTER TABLE mt_00763_1 MODIFY column s String;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error ||: } function thread2() { - seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM test.buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' + seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' } thread1 & @@ -28,5 +28,5 @@ thread2 & wait -${CLICKHOUSE_CLIENT} --query="DROP TABLE test.mt_00763_1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE test.buffer_00763_1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mt_00763_1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE buffer_00763_1" diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql index 2e1df541f14..9f0850ff7e5 100644 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql @@ -1,15 +1,15 @@ -DROP TABLE IF EXISTS test.part_header_r1; -DROP TABLE IF EXISTS test.part_header_r2; +DROP TABLE IF EXISTS part_header_r1; +DROP TABLE IF EXISTS part_header_r2; SET replication_alter_partitions_sync = 2; -CREATE TABLE test.part_header_r1(x UInt32, y UInt32) +CREATE TABLE part_header_r1(x UInt32, y UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/part_header', '1') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 0, old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0; -CREATE TABLE test.part_header_r2(x UInt32, y UInt32) +CREATE TABLE part_header_r2(x UInt32, y UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/part_header', '2') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 1, old_parts_lifetime = 1, @@ -17,47 +17,47 @@ CREATE TABLE test.part_header_r2(x UInt32, y UInt32) cleanup_delay_period_random_add = 0; SELECT '*** Test fetches ***'; -INSERT INTO test.part_header_r1 VALUES (1, 1); -INSERT INTO test.part_header_r2 VALUES (2, 2); -SYSTEM SYNC REPLICA test.part_header_r1; -SYSTEM SYNC REPLICA test.part_header_r2; +INSERT INTO part_header_r1 VALUES (1, 1); +INSERT INTO part_header_r2 VALUES (2, 2); +SYSTEM SYNC REPLICA part_header_r1; +SYSTEM SYNC REPLICA part_header_r2; SELECT '*** replica 1 ***'; -SELECT x, y FROM test.part_header_r1 ORDER BY x; +SELECT x, y FROM part_header_r1 ORDER BY x; SELECT '*** replica 2 ***'; -SELECT x, y FROM test.part_header_r2 ORDER BY x; +SELECT x, y FROM part_header_r2 ORDER BY x; SELECT '*** Test merges ***'; -OPTIMIZE TABLE test.part_header_r1; -SYSTEM SYNC REPLICA test.part_header_r2; +OPTIMIZE TABLE part_header_r1; +SYSTEM SYNC REPLICA part_header_r2; SELECT '*** replica 1 ***'; -SELECT _part, x FROM test.part_header_r1 ORDER BY x; +SELECT _part, x FROM part_header_r1 ORDER BY x; SELECT '*** replica 2 ***'; -SELECT _part, x FROM test.part_header_r2 ORDER BY x; +SELECT _part, x FROM part_header_r2 ORDER BY x; SELECT sleep(3) FORMAT Null; SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; -SELECT name FROM system.parts WHERE active AND database = 'test' AND table = 'part_header_r1'; +SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1'; SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test/part_header/replicas/1/parts'; SELECT '*** replica 2 ***'; -SELECT name FROM system.parts WHERE active AND database = 'test' AND table = 'part_header_r2'; +SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2'; SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test/part_header/replicas/1/parts'; SELECT '*** Test ALTER ***'; -ALTER TABLE test.part_header_r1 MODIFY COLUMN y String; +ALTER TABLE part_header_r1 MODIFY COLUMN y String; SELECT '*** replica 1 ***'; -SELECT x, length(y) FROM test.part_header_r1 ORDER BY x; +SELECT x, length(y) FROM part_header_r1 ORDER BY x; SELECT '*** replica 2 ***'; -SELECT x, length(y) FROM test.part_header_r2 ORDER BY x; +SELECT x, length(y) FROM part_header_r2 ORDER BY x; SELECT '*** Test CLEAR COLUMN ***'; SET replication_alter_partitions_sync = 2; -ALTER TABLE test.part_header_r1 CLEAR COLUMN y IN PARTITION tuple(); +ALTER TABLE part_header_r1 CLEAR COLUMN y IN PARTITION tuple(); SELECT '*** replica 1 ***'; -SELECT x, length(y) FROM test.part_header_r1 ORDER BY x; +SELECT x, length(y) FROM part_header_r1 ORDER BY x; SELECT '*** replica 2 ***'; -SELECT x, length(y) FROM test.part_header_r2 ORDER BY x; +SELECT x, length(y) FROM part_header_r2 ORDER BY x; -DROP TABLE test.part_header_r1; -DROP TABLE test.part_header_r2; +DROP TABLE part_header_r1; +DROP TABLE part_header_r2; diff --git a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql index 5b785d31497..a7ae0402b5a 100644 --- a/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql +++ b/tests/queries/0_stateless/00926_zookeeper_adaptive_index_granularity_replicated_merge_tree.sql @@ -1,214 +1,214 @@ ----- Group of very similar simple tests ------ select '----HORIZONTAL MERGE TESTS----'; -DROP TABLE IF EXISTS test.zero_rows_per_granule1; -DROP TABLE IF EXISTS test.zero_rows_per_granule2; +DROP TABLE IF EXISTS zero_rows_per_granule1; +DROP TABLE IF EXISTS zero_rows_per_granule2; -CREATE TABLE test.zero_rows_per_granule1 ( +CREATE TABLE zero_rows_per_granule1 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/zero_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0; -CREATE TABLE test.zero_rows_per_granule2 ( +CREATE TABLE zero_rows_per_granule2 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/zero_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 20, write_final_mark = 0; -INSERT INTO test.zero_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +INSERT INTO zero_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -SYSTEM SYNC REPLICA test.zero_rows_per_granule2; +SYSTEM SYNC REPLICA zero_rows_per_granule2; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.zero_rows_per_granule1; +SELECT COUNT(*) FROM zero_rows_per_granule1; -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database=currentDatabase() and active=1; -SELECT COUNT(*) FROM test.zero_rows_per_granule2; +SELECT COUNT(*) FROM zero_rows_per_granule2; -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database=currentDatabase() and active=1; -INSERT INTO test.zero_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000); +INSERT INTO zero_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000); -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database=currentDatabase() and active=1; -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database=currentDatabase() and active=1; SELECT sleep(0.7) Format Null; -OPTIMIZE TABLE test.zero_rows_per_granule2 FINAL; +OPTIMIZE TABLE zero_rows_per_granule2 FINAL; SELECT 'Parts optimized'; -SYSTEM SYNC REPLICA test.zero_rows_per_granule1; +SYSTEM SYNC REPLICA zero_rows_per_granule1; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.zero_rows_per_granule2; +SELECT COUNT(*) FROM zero_rows_per_granule2; -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule2' and database=currentDatabase() and active=1; -SELECT COUNT(*) FROM test.zero_rows_per_granule1; +SELECT COUNT(*) FROM zero_rows_per_granule1; -SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'zero_rows_per_granule1' and database=currentDatabase() and active=1; -DROP TABLE IF EXISTS test.zero_rows_per_granule1; -DROP TABLE IF EXISTS test.zero_rows_per_granule2; +DROP TABLE IF EXISTS zero_rows_per_granule1; +DROP TABLE IF EXISTS zero_rows_per_granule2; SELECT '-----'; -DROP TABLE IF EXISTS test.four_rows_per_granule1; -DROP TABLE IF EXISTS test.four_rows_per_granule2; +DROP TABLE IF EXISTS four_rows_per_granule1; +DROP TABLE IF EXISTS four_rows_per_granule2; -CREATE TABLE test.four_rows_per_granule1 ( +CREATE TABLE four_rows_per_granule1 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/four_rows_in_granule', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; -CREATE TABLE test.four_rows_per_granule2 ( +CREATE TABLE four_rows_per_granule2 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/four_rows_in_granule', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; -INSERT INTO test.four_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +INSERT INTO four_rows_per_granule1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -SELECT COUNT(*) FROM test.four_rows_per_granule1; +SELECT COUNT(*) FROM four_rows_per_granule1; -SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.four_rows_per_granule2; +SYSTEM SYNC REPLICA four_rows_per_granule2; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.four_rows_per_granule2; +SELECT COUNT(*) FROM four_rows_per_granule2; -SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database=currentDatabase() and active=1; -DETACH TABLE test.four_rows_per_granule2; -ATTACH TABLE test.four_rows_per_granule2; +DETACH TABLE four_rows_per_granule2; +ATTACH TABLE four_rows_per_granule2; SELECT 'Table attached'; -INSERT INTO test.four_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000); +INSERT INTO four_rows_per_granule2 (p, k, v1, v2) VALUES ('2018-05-15', 5, 1000, 2000), ('2018-05-16', 6, 3000, 4000), ('2018-05-17', 7, 5000, 6000), ('2018-05-19', 8, 7000, 8000); -SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database=currentDatabase() and active=1; -SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database=currentDatabase() and active=1; SELECT sleep(0.7) Format Null; -OPTIMIZE TABLE test.four_rows_per_granule2 FINAL; +OPTIMIZE TABLE four_rows_per_granule2 FINAL; SELECT 'Parts optimized'; -DETACH TABLE test.four_rows_per_granule2; +DETACH TABLE four_rows_per_granule2; -ATTACH TABLE test.four_rows_per_granule2; +ATTACH TABLE four_rows_per_granule2; -SELECT COUNT(*) FROM test.four_rows_per_granule2; +SELECT COUNT(*) FROM four_rows_per_granule2; ---SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database='test' and active=1; +--SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule2' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.four_rows_per_granule1; +SYSTEM SYNC REPLICA four_rows_per_granule1; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.four_rows_per_granule1; +SELECT COUNT(*) FROM four_rows_per_granule1; ---SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database='test' and active=1; +--SELECT distinct(marks) from system.parts WHERE table = 'four_rows_per_granule1' and database=currentDatabase() and active=1; -DROP TABLE IF EXISTS test.four_rows_per_granule1; -DROP TABLE IF EXISTS test.four_rows_per_granule2; +DROP TABLE IF EXISTS four_rows_per_granule1; +DROP TABLE IF EXISTS four_rows_per_granule2; SELECT '-----'; -DROP TABLE IF EXISTS test.adaptive_granularity_alter1; -DROP TABLE IF EXISTS test.adaptive_granularity_alter2; +DROP TABLE IF EXISTS adaptive_granularity_alter1; +DROP TABLE IF EXISTS adaptive_granularity_alter2; -CREATE TABLE test.adaptive_granularity_alter1 ( +CREATE TABLE adaptive_granularity_alter1 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/adaptive_granularity_alter', '1') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; -CREATE TABLE test.adaptive_granularity_alter2 ( +CREATE TABLE adaptive_granularity_alter2 ( p Date, k UInt64, v1 UInt64, v2 Int64 ) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/adaptive_granularity_alter', '2') PARTITION BY toYYYYMM(p) ORDER BY k SETTINGS index_granularity_bytes = 110, write_final_mark = 0; -INSERT INTO test.adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); +INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 2, 3000, 4000), ('2018-05-17', 3, 5000, 6000), ('2018-05-18', 4, 7000, 8000); -SELECT COUNT(*) FROM test.adaptive_granularity_alter1; +SELECT COUNT(*) FROM adaptive_granularity_alter1; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.adaptive_granularity_alter2; +SYSTEM SYNC REPLICA adaptive_granularity_alter2; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.adaptive_granularity_alter2; +SELECT COUNT(*) FROM adaptive_granularity_alter2; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1; -ALTER TABLE test.adaptive_granularity_alter2 MODIFY COLUMN v1 Int16; +ALTER TABLE adaptive_granularity_alter2 MODIFY COLUMN v1 Int16; -DETACH TABLE test.adaptive_granularity_alter2; +DETACH TABLE adaptive_granularity_alter2; -ATTACH TABLE test.adaptive_granularity_alter2; +ATTACH TABLE adaptive_granularity_alter2; -SELECT COUNT(*) FROM test.adaptive_granularity_alter2; +SELECT COUNT(*) FROM adaptive_granularity_alter2; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.adaptive_granularity_alter1; +SYSTEM SYNC REPLICA adaptive_granularity_alter1; SELECT 'Replica synced'; -SELECT COUNT(*) FROM test.adaptive_granularity_alter1; +SELECT COUNT(*) FROM adaptive_granularity_alter1; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1; -INSERT INTO test.adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 5, 3000, 4000), ('2018-05-17', 6, 5000, 6000), ('2018-05-19', 42, 42, 42); +INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 1, 1000, 2000), ('2018-05-16', 5, 3000, 4000), ('2018-05-17', 6, 5000, 6000), ('2018-05-19', 42, 42, 42); -SELECT COUNT(*) FROM test.adaptive_granularity_alter1; +SELECT COUNT(*) FROM adaptive_granularity_alter1; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.adaptive_granularity_alter2; +SYSTEM SYNC REPLICA adaptive_granularity_alter2; -SELECT COUNT(*) FROM test.adaptive_granularity_alter2; +SELECT COUNT(*) FROM adaptive_granularity_alter2; -SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1; +SELECT distinct(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1; -ALTER TABLE test.adaptive_granularity_alter1 MODIFY COLUMN v2 String; +ALTER TABLE adaptive_granularity_alter1 MODIFY COLUMN v2 String; -DETACH TABLE test.adaptive_granularity_alter1; +DETACH TABLE adaptive_granularity_alter1; -ATTACH TABLE test.adaptive_granularity_alter1; +ATTACH TABLE adaptive_granularity_alter1; -INSERT INTO test.adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 100, 1000, 'aaaa'), ('2018-05-16', 101, 3000, 'bbbb'), ('2018-05-17', 102, 5000, 'cccc'), ('2018-05-19', 103, 7000, 'dddd'); +INSERT INTO adaptive_granularity_alter1 (p, k, v1, v2) VALUES ('2018-05-15', 100, 1000, 'aaaa'), ('2018-05-16', 101, 3000, 'bbbb'), ('2018-05-17', 102, 5000, 'cccc'), ('2018-05-19', 103, 7000, 'dddd'); SELECT sleep(0.7) Format Null; -OPTIMIZE TABLE test.adaptive_granularity_alter1 FINAL; +OPTIMIZE TABLE adaptive_granularity_alter1 FINAL; SELECT 'Parts optimized'; -SELECT k, v2 FROM test.adaptive_granularity_alter1 WHERE k >= 100 OR k = 42 ORDER BY k; +SELECT k, v2 FROM adaptive_granularity_alter1 WHERE k >= 100 OR k = 42 ORDER BY k; -SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database='test' and active=1; +SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter1' and database=currentDatabase() and active=1; -SYSTEM SYNC REPLICA test.adaptive_granularity_alter2; +SYSTEM SYNC REPLICA adaptive_granularity_alter2; SELECT 'Replica synced'; -SELECT k, v2 FROM test.adaptive_granularity_alter2 WHERE k >= 100 OR k = 42 ORDER BY k; +SELECT k, v2 FROM adaptive_granularity_alter2 WHERE k >= 100 OR k = 42 ORDER BY k; -SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database='test' and active=1; +SELECT sum(marks) from system.parts WHERE table = 'adaptive_granularity_alter2' and database=currentDatabase() and active=1; -DROP TABLE IF EXISTS test.adaptive_granularity_alter1; -DROP TABLE IF EXISTS test.adaptive_granularity_alter2; +DROP TABLE IF EXISTS adaptive_granularity_alter1; +DROP TABLE IF EXISTS adaptive_granularity_alter2; diff --git a/tests/queries/0_stateless/00937_test_use_header_csv.sh b/tests/queries/0_stateless/00937_test_use_header_csv.sh index bd9204ffdcc..c0d4f2be522 100755 --- a/tests/queries/0_stateless/00937_test_use_header_csv.sh +++ b/tests/queries/0_stateless/00937_test_use_header_csv.sh @@ -3,10 +3,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.csv" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.csv (d Date, u UInt8, str String) ENGINE = TinyLog" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS csv" +$CLICKHOUSE_CLIENT --query="CREATE TABLE csv (d Date, u UInt8, str String) ENGINE = TinyLog" -INSERT_QUERY='$CLICKHOUSE_CLIENT --query="INSERT INTO test.csv FORMAT CSVWithNames"' +INSERT_QUERY='$CLICKHOUSE_CLIENT --query="INSERT INTO csv FORMAT CSVWithNames"' USE_HEADER='--input_format_with_names_use_header=1' SKIP_UNKNOWN='--input_format_skip_unknown_fields=1' @@ -32,5 +32,5 @@ echo -ne 'str,u\nLine16,1\nLine17,2\n' | eval $INSERT_QUERY $US echo -ne 'd,str\n2019-04-18,Line18\n2019-04-18,Line19\n'| eval $INSERT_QUERY $USE_HEADER echo -ne 'unknown\n\n\n' | eval $INSERT_QUERY $USE_HEADER $SKIP_UNKNOWN -$CLICKHOUSE_CLIENT --query="SELECT * FROM test.csv" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.csv" +$CLICKHOUSE_CLIENT --query="SELECT * FROM csv" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS csv" diff --git a/tests/queries/0_stateless/00942_mutate_index.sh b/tests/queries/0_stateless/00942_mutate_index.sh index 30ac7e8821b..d14c5659fc9 100755 --- a/tests/queries/0_stateless/00942_mutate_index.sh +++ b/tests/queries/0_stateless/00942_mutate_index.sh @@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh . $CURDIR/mergetree_mutations.lib -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" $CLICKHOUSE_CLIENT -n --query=" -CREATE TABLE test.minmax_idx +CREATE TABLE minmax_idx ( u64 UInt64, i64 Int64, @@ -19,7 +19,7 @@ ORDER BY u64 SETTINGS index_granularity = 2;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES +$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES (0, 1, 1), (1, 1, 2), (2, 1, 3), @@ -31,13 +31,13 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES (8, 1, 9), (9, 1, 10)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 1;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 5;" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx UPDATE i64 = 5 WHERE i64 = 1;" -wait_for_mutation "minmax_idx" "mutation_2.txt" "test" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx UPDATE i64 = 5 WHERE i64 = 1;" +wait_for_mutation "minmax_idx" "mutation_2.txt" "$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 1;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 5;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 1;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 5;" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" +$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx" diff --git a/tests/queries/0_stateless/00943_materialize_index.sh b/tests/queries/0_stateless/00943_materialize_index.sh index b406f3894eb..c3d5a6bb219 100755 --- a/tests/queries/0_stateless/00943_materialize_index.sh +++ b/tests/queries/0_stateless/00943_materialize_index.sh @@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh . $CURDIR/mergetree_mutations.lib -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" $CLICKHOUSE_CLIENT -n --query=" -CREATE TABLE test.minmax_idx +CREATE TABLE minmax_idx ( u64 UInt64, i64 Int64, @@ -19,7 +19,7 @@ ORDER BY u64 SETTINGS index_granularity = 2;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES +$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES (0, 2, 1), (1, 1, 1), (2, 1, 1), @@ -31,35 +31,35 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES (8, 1, 2), (9, 1, 2)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT -n --query=" -ALTER TABLE test.minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;" +ALTER TABLE minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1;" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" -wait_for_mutation "minmax_idx" "mutation_3.txt" "test" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" +wait_for_mutation "minmax_idx" "mutation_3.txt" "$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;" -wait_for_mutation "minmax_idx" "mutation_4.txt" "test" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 2;" +wait_for_mutation "minmax_idx" "mutation_4.txt" "$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 2;" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 1;" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 2;" sleep 0.5 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx;" -wait_for_mutation "minmax_idx" "mutation_5.txt" "test" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx;" +wait_for_mutation "minmax_idx" "mutation_5.txt" "$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" +$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx" diff --git a/tests/queries/0_stateless/00944_clear_index_in_partition.sh b/tests/queries/0_stateless/00944_clear_index_in_partition.sh index 9f53ec16c35..d6ca4106a0e 100755 --- a/tests/queries/0_stateless/00944_clear_index_in_partition.sh +++ b/tests/queries/0_stateless/00944_clear_index_in_partition.sh @@ -4,11 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh . $CURDIR/mergetree_mutations.lib -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.minmax_idx;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS minmax_idx;" $CLICKHOUSE_CLIENT -n --query=" -CREATE TABLE test.minmax_idx +CREATE TABLE minmax_idx ( u64 UInt64, i64 Int64, @@ -20,7 +20,7 @@ ORDER BY u64 SETTINGS index_granularity = 2;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES +$CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES (0, 2, 1), (1, 1, 1), (2, 1, 1), @@ -32,19 +32,19 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.minmax_idx VALUES (8, 1, 2), (9, 1, 2)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" -wait_for_mutation "minmax_idx" "mutation_3.txt" "test" +$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" +wait_for_mutation "minmax_idx" "mutation_3.txt" "$CLICKHOUSE_DATABASE" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.minmax_idx" +$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx" diff --git a/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh b/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh index 28e7bd0b94e..b8b133cde3d 100755 --- a/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/00975_indices_mutation_replicated_zookeeper.sh @@ -4,12 +4,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh . $CURDIR/mergetree_mutations.lib -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions1;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.indices_mutaions2;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS indices_mutaions1;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS indices_mutaions2;" $CLICKHOUSE_CLIENT -n --query=" -CREATE TABLE test.indices_mutaions1 +CREATE TABLE indices_mutaions1 ( u64 UInt64, i64 Int64, @@ -19,7 +19,7 @@ CREATE TABLE test.indices_mutaions1 PARTITION BY i32 ORDER BY u64 SETTINGS index_granularity = 2; -CREATE TABLE test.indices_mutaions2 +CREATE TABLE indices_mutaions2 ( u64 UInt64, i64 Int64, @@ -31,7 +31,7 @@ ORDER BY u64 SETTINGS index_granularity = 2;" -$CLICKHOUSE_CLIENT --query="INSERT INTO test.indices_mutaions1 VALUES +$CLICKHOUSE_CLIENT --query="INSERT INTO indices_mutaions1 VALUES (0, 2, 1), (1, 1, 1), (2, 1, 1), @@ -43,18 +43,18 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO test.indices_mutaions1 VALUES (8, 1, 2), (9, 1, 2)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2 +$CLICKHOUSE_CLIENT --query="ALTER TABLE indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="ALTER TABLE test.indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2 +$CLICKHOUSE_CLIENT --query="ALTER TABLE indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2;" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM test.indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions1" -$CLICKHOUSE_CLIENT --query="DROP TABLE test.indices_mutaions2" +$CLICKHOUSE_CLIENT --query="DROP TABLE indices_mutaions1" +$CLICKHOUSE_CLIENT --query="DROP TABLE indices_mutaions2" diff --git a/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh b/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh index 19d2b487955..067c19658ab 100755 --- a/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh +++ b/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh @@ -3,17 +3,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.a;" -$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test.b;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS a;" +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS b;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.a (x UInt64) ENGINE = Memory;" -$CLICKHOUSE_CLIENT --query="CREATE TABLE test.b (x UInt64) ENGINE = Memory;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE a (x UInt64) ENGINE = Memory;" +$CLICKHOUSE_CLIENT --query="CREATE TABLE b (x UInt64) ENGINE = Memory;" function thread1() { for attempt_thread1 in {1..10} do - seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=11 --query="INSERT INTO test.a FORMAT TSV" & + seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=11 --query="INSERT INTO a FORMAT TSV" & while true; do $CLICKHOUSE_CLIENT --query="KILL QUERY WHERE query_id='11' SYNC" | grep -q "cant_cancel" && sleep .1 || break ||: done @@ -27,7 +27,7 @@ function thread2() { for attempt_thread2 in {1..10} do - seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=22 --query="INSERT INTO test.b FORMAT TSV" & + seq 1 500000 | $CLICKHOUSE_CLIENT --query_id=22 --query="INSERT INTO b FORMAT TSV" & while true; do $CLICKHOUSE_CLIENT --query="KILL QUERY WHERE query_id='22' SYNC" | grep -q "cant_cancel" && sleep .1 || break ||: done @@ -46,5 +46,5 @@ bash -c thread2 > /dev/null 2>&1 & wait echo OK -$CLICKHOUSE_CLIENT --query "DROP TABLE test.a" -$CLICKHOUSE_CLIENT --query "DROP TABLE test.b" +$CLICKHOUSE_CLIENT --query "DROP TABLE a" +$CLICKHOUSE_CLIENT --query "DROP TABLE b" diff --git a/tests/queries/0_stateless/01122_totals_rollup_having_block_header.sql b/tests/queries/0_stateless/01122_totals_rollup_having_block_header.sql index 4f4f3355912..6fb877c350a 100644 --- a/tests/queries/0_stateless/01122_totals_rollup_having_block_header.sql +++ b/tests/queries/0_stateless/01122_totals_rollup_having_block_header.sql @@ -1,14 +1,14 @@ -DROP TABLE IF EXISTS test.rollup_having; -CREATE TABLE test.rollup_having ( +DROP TABLE IF EXISTS rollup_having; +CREATE TABLE rollup_having ( a Nullable(String), b Nullable(String) ) ENGINE = Memory; -INSERT INTO test.rollup_having VALUES (NULL, NULL); -INSERT INTO test.rollup_having VALUES ('a', NULL); -INSERT INTO test.rollup_having VALUES ('a', 'b'); +INSERT INTO rollup_having VALUES (NULL, NULL); +INSERT INTO rollup_having VALUES ('a', NULL); +INSERT INTO rollup_having VALUES ('a', 'b'); -SELECT a, b, count(*) FROM test.rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL; -- { serverError 48 } -SELECT a, b, count(*) FROM test.rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL and b IS NOT NULL; -- { serverError 48 } +SELECT a, b, count(*) FROM rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL; -- { serverError 48 } +SELECT a, b, count(*) FROM rollup_having GROUP BY a, b WITH ROLLUP WITH TOTALS HAVING a IS NOT NULL and b IS NOT NULL; -- { serverError 48 } -DROP TABLE test.rollup_having; +DROP TABLE rollup_having; diff --git a/tests/queries/0_stateless/01124_view_bad_types.sql b/tests/queries/0_stateless/01124_view_bad_types.sql index 81fc53930c1..715f7b375f1 100644 --- a/tests/queries/0_stateless/01124_view_bad_types.sql +++ b/tests/queries/0_stateless/01124_view_bad_types.sql @@ -1,11 +1,11 @@ -DROP TABLE IF EXISTS test.table; -CREATE TABLE test.table (x UInt16) ENGINE = TinyLog; -INSERT INTO test.table SELECT * FROM system.numbers LIMIT 10; +DROP TABLE IF EXISTS source_table; +CREATE TABLE source_table (x UInt16) ENGINE = TinyLog; +INSERT INTO source_table SELECT * FROM system.numbers LIMIT 10; -DROP TABLE IF EXISTS test.view; -CREATE VIEW test.view (x UInt64) AS SELECT * FROM test.table; +DROP TABLE IF EXISTS dest_view; +CREATE VIEW dest_view (x UInt64) AS SELECT * FROM source_table; -SELECT x, any(x) FROM test.view GROUP BY x ORDER BY x; +SELECT x, any(x) FROM dest_view GROUP BY x ORDER BY x; -DROP TABLE test.view; -DROP TABLE test.table; +DROP TABLE dest_view; +DROP TABLE source_table; diff --git a/tests/queries/1_stateful/00155_date_datetime_key_condition.reference b/tests/queries/0_stateless/01285_date_datetime_key_condition.reference similarity index 100% rename from tests/queries/1_stateful/00155_date_datetime_key_condition.reference rename to tests/queries/0_stateless/01285_date_datetime_key_condition.reference diff --git a/tests/queries/0_stateless/01285_date_datetime_key_condition.sql b/tests/queries/0_stateless/01285_date_datetime_key_condition.sql new file mode 100644 index 00000000000..fe1454cd7c4 --- /dev/null +++ b/tests/queries/0_stateless/01285_date_datetime_key_condition.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS date_datetime_key_condition; + +CREATE TABLE date_datetime_key_condition (dt DateTime) ENGINE = MergeTree() ORDER BY dt; +INSERT INTO date_datetime_key_condition VALUES ('2020-01-01 00:00:00'), ('2020-01-01 10:00:00'), ('2020-01-02 00:00:00'); + +-- partial +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2020-01-01') AND dt < toDate('2020-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt >= toDate('2020-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2020-01-02'); + +-- inside +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2019-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2021-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt >= toDate('2019-01-02') AND dt < toDate('2021-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2019-01-02') OR dt <= toDate('2021-01-02'); + +-- outside +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2019-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt > toDate('2021-01-02'); +SELECT groupArray(dt) from date_datetime_key_condition WHERE dt < toDate('2019-01-02') OR dt > toDate('2021-01-02'); + +DROP TABLE date_datetime_key_condition; diff --git a/tests/queries/1_stateful/00155_date_datetime_key_condition.sql b/tests/queries/1_stateful/00155_date_datetime_key_condition.sql deleted file mode 100644 index cee1c7cfb46..00000000000 --- a/tests/queries/1_stateful/00155_date_datetime_key_condition.sql +++ /dev/null @@ -1,22 +0,0 @@ -DROP TABLE IF EXISTS test.date_datetime_key_condition; - -CREATE TABLE test.date_datetime_key_condition (dt DateTime) ENGINE = MergeTree() ORDER BY dt; -INSERT INTO test.date_datetime_key_condition VALUES ('2020-01-01 00:00:00'), ('2020-01-01 10:00:00'), ('2020-01-02 00:00:00'); - --- partial -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2020-01-01') AND dt < toDate('2020-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt >= toDate('2020-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2020-01-02'); - --- inside -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2019-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2021-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt >= toDate('2019-01-02') AND dt < toDate('2021-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2019-01-02') OR dt <= toDate('2021-01-02'); - --- outside -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2019-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt > toDate('2021-01-02'); -SELECT groupArray(dt) from test.date_datetime_key_condition WHERE dt < toDate('2019-01-02') OR dt > toDate('2021-01-02'); - -DROP TABLE test.date_datetime_key_condition; \ No newline at end of file From bbb57b77893a38f363707d1bdf9540214a839dd0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 May 2020 13:31:19 +0300 Subject: [PATCH 067/150] Try fix tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9b161bc1883..56532e532d1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -73,6 +73,7 @@ #include #include #include +#include namespace DB @@ -443,6 +444,16 @@ BlockIO InterpreterSelectQuery::execute() executeImpl(res.pipeline, input, std::move(input_pipe)); res.pipeline.addInterpreterContext(context); res.pipeline.addStorageHolder(storage); + + /// We must guarantee that result structure is the same as in getSampleBlock() + if (!blocksHaveEqualStructure(res.pipeline.getHeader(), result_header)) + { + res.pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, result_header, ConvertingTransform::MatchColumnsMode::Name); + }); + } + return res; } From 2ff7bbb5115270864beadf2fce71e497858903a1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 29 May 2020 13:56:04 +0300 Subject: [PATCH 068/150] Better names --- src/Storages/TTLDescription.cpp | 2 +- src/Storages/TTLDescription.h | 13 +++++++------ 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 92fcf400dd3..da9691aab4a 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -180,7 +180,7 @@ TTLDescription TTLDescription::getTTLFromAST( auto syntax_result = SyntaxAnalyzer(context).analyze(value, columns.getAllPhysical(), {}, true); auto expr_analyzer = ExpressionAnalyzer(value, syntax_result, context); - result.set_parts.emplace_back(TTLSetPartDescription{ + result.set_parts.emplace_back(TTLAggregateDescription{ name, value->getColumnName(), expr_analyzer.getActions(false)}); for (const auto & descr : expr_analyzer.getAnalyzedData().aggregate_descriptions) diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index d0e669ef4cf..99a145b8acc 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -10,23 +10,24 @@ namespace DB { -struct TTLSetPartDescription +/// Assignment expression in TTL with GROUP BY +struct TTLAggregateDescription { - /// Name of column in set part of ttl expression + /// Name of column in assignment /// x = sum(y) /// ^ String column_name; - /// Name of column on the right hand of the set part of TTL expression + /// Name of column on the right hand of the assignment /// x = sum(y) /// ^~~~~~^ String expression_result_column_name; - /// Expressions to calculate the value of set expression + /// Expressions to calculate the value of assignment expression ExpressionActionsPtr expression; }; -using TTLSetPartDescriptions = std::vector; +using TTLAggregateDescriptions = std::vector; /// Common struct for TTL record in storage struct TTLDescription @@ -58,7 +59,7 @@ struct TTLDescription Names group_by_keys; /// SET parts of TTL expression - TTLSetPartDescriptions set_parts; + TTLAggregateDescriptions set_parts; /// Aggregate descriptions for GROUP BY in TTL AggregateDescriptions aggregate_descriptions; From 889c7889593e8478830a483a971fc4029687d065 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 29 May 2020 15:49:37 +0300 Subject: [PATCH 069/150] better --- CHANGELOG.md | 74 +++++++++++++++++++++++++--------------------------- 1 file changed, 35 insertions(+), 39 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6cbf3d96baf..5318a5adcec 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -32,12 +32,12 @@ * Fixed combinator `-OrNull` and `-OrDefault` when combined with `-State`. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). * Fixed possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fixed bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). -* Fixed `nullpt`r dereference in `StorageBuffer` if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `nullptr` dereference in `StorageBuffer` if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fixed `optimize_skip_unused_shards` with `LowCardinality`. [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). * Fixed handling condition variable for synchronous mutations. In some cases signals to that condition variable could be lost. [#10588](https://github.com/ClickHouse/ClickHouse/pull/10588) ([Vladimir Chebotarev](https://github.com/excitoon)). * Fixed possible crash when `createDictionary()` is called before `loadStoredObject()` has finished. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). * Fixed `SELECT` of column `ALIAS` which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* Fixed implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). +* Implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). * Disable `GROUP BY` sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for `WITH ROLLUP/CUBE/TOTALS`. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). * Fixed [#10263] (https://github.com/ClickHouse/ClickHouse/issues/10263). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). * Added failing tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). @@ -327,51 +327,47 @@ #### Bug Fix -* Remove logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). -* Fixed parseDateTime64BestEffort argument resolution bugs. ... [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). -* Fix incorrect raw data size in method getRawData(). [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). -* Fix incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). -* Fix SIGSEGV in StringHashTable (if such key does not exist). [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). +* Removed logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). +* Fixed `parseDateTime64BestEffort` argument resolution bugs. [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). +* Fixed incorrect raw data size in method `getRawData()`. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). +* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of `GROUP BY` result is large and aggregation is performed by a single `String` field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed backward compatibility with tuples in `Distributed` tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed `SIGSEGV` in `StringHashTable` if such a key does not exist. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). * Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). -* Fix columns order after Block::sortColumns() (also add a test that shows that it affects some real use case - Buffer engine). [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). -* Fix the issue with ODBC bridge when no quoting of identifiers is requested. This fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix UBSan and MSan report in DateLUT. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* -. [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). -* Fix `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix combinator -OrNull and -OrDefault when combined with -State. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). -* Fix crash in `generateRandom` with nested types. Fixes [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix disappearing totals. Totals could have being filtered if query had had join or subquery with external where condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix multiple usages of `IN` operator with the identical set in one query. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed columns order after `Block::sortColumns()`. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the issue with `ODBC` bridge when no quoting of identifiers is requested. Fixes [#7984] (https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `UBSan` and `MSan` report in `DateLUT`. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)) +* Fixed `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed combinator -`OrNull` and `-OrDefault` when combined with `-State`. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). +* Fixed crash in `generateRandom` with nested types. Fixes [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed disappearing totals. Totals could have being filtered if query had had join or subquery with external where condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed multiple usages of `IN` operator with the identical set in one query. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). * Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). -* Fix order of parameters in AggregateTransform constructor. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). -* Fix the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix possible incorrect number of rows for queries with `LIMIT`. Fixes [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709). [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). -* On `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). +* Fixed order of parameters in `AggregateTransform` constructor. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). +* Fixed the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible incorrect number of rows for queries with `LIMIT`. Fixes [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709). [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed a bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). +* Fixed a bug when on `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). * Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix SELECT of column ALIAS which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* * implemented comparison between DateTime64 and String values (just like for DateTime). ... [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). -* Fix index corruption, which may accur in some cases after merge compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)). -* Now the situation, when mutation finished all parts, but hung up in `is_done=0`, is impossible. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). -* Fix overflow at beginning of unix epoch for timezones with fractional offset from UTC. This fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Proper Distributed shutdown (fixes UAF, avoid waiting for sending all batches). [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). -* add CA certificates to clickhouse-server docker image. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). -* Fix numeric overflow in simpleLinearRegression over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). -* Minor improvements in printing diagnostic info in input formats. Fixes [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204). [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)). -* Todo. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `SELECT` of column `ALIAS` which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). +* Implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). +* Fixed index corruption, which may accur in some cases after merge compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed the situation, when mutation finished all parts, but hung up in `is_done=0`. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). +* Fixed overflow at beginning of unix epoch for timezones with fractional offset from `UTC`. This fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed improper shutdown of `Distributed` storage. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). +* Fixed numeric overflow in `simpleLinearRegression` over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). + #### Build/Testing/Packaging Improvement * Fix UBSan report in LZ4 library. [#10631](https://github.com/ClickHouse/ClickHouse/pull/10631) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix clang-10 build. https://github.com/ClickHouse/ClickHouse/issues/10238. [#10370](https://github.com/ClickHouse/ClickHouse/pull/10370) ([Amos Bird](https://github.com/amosbird)). - -#### NO CL ENTRY - -* NO CL ENTRY: '20.3 backports part 2'. [#11157](https://github.com/ClickHouse/ClickHouse/pull/11157) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* NO CL ENTRY: '20.3 backports part 1'. [#11018](https://github.com/ClickHouse/ClickHouse/pull/11018) ([Ivan](https://github.com/abyss7)). +* Added failing tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added some improvements in printing diagnostic info in input formats. Fixes [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204). [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)). +* Added CA certificates to clickhouse-server docker image. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). #### Bug fix From bd814d83de02abdb6b7bda79bde173cc54835f92 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 May 2020 16:04:03 +0300 Subject: [PATCH 070/150] Try fix tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 56532e532d1..fb3188545f1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -450,7 +450,7 @@ BlockIO InterpreterSelectQuery::execute() { res.pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, result_header, ConvertingTransform::MatchColumnsMode::Name); + return std::make_shared(header, result_header, ConvertingTransform::MatchColumnsMode::Position); }); } From 341a73507548fcb2733494ffaf243ae041b2adc8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 29 May 2020 16:17:12 +0300 Subject: [PATCH 071/150] 20.1 --- CHANGELOG.md | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5318a5adcec..14638636d2b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -724,6 +724,35 @@ ## ClickHouse release v20.1 +### ClickHouse release v20.1.12.86, 2020-05-26 + +#### Bug Fix + +* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). +* Fixed a bug when on `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). +* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the situation when mutation finished all parts, but hung up in `is_done=0`. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). +* Fixed overflow at beginning of unix epoch for timezones with fractional offset from UTC. This fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed improper shutdown of Distributed storage. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). +* Fixed numeric overflow in `simpleLinearRegression` over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). +* Fixed removing metadata directory when attach database fails. [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)). +* Added a check of number and type of arguments when creating BloomFilter index [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623). [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed the issue when a query with ARRAY JOIN, ORDER BY and LIMIT may return incomplete result. This fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). Author: [Vadim Plakhtinskiy](https://github.com/VadimPlh). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Prefer `fallback_to_stale_replicas` over `skip_unavailable_shards`. [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong flattening of `Array(Tuple(...))` data types. This fixes [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259). [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong behavior in `HashTable` that caused compilation error when trying to read HashMap from buffer. [#10386](https://github.com/ClickHouse/ClickHouse/pull/10386) ([palasonic1](https://github.com/palasonic1)). +* Fixed possible `Pipeline stuck` error in `ConcatProcessor` which could have happened in remote query. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Pipeline stuck` with `max_rows_to_group_by` and `group_by_overflow_mode = 'break'`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed several bugs when some data was inserted with quorum, then deleted somehow (DROP PARTITION, TTL) and this leaded to the stuck of INSERTs or false-positive exceptions in SELECTs. This fixes [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946). [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed incompatibility when versions prior to 18.12.17 are used on remote servers and newer is used on initiating server, and GROUP BY both fixed and non-fixed keys, and when two-level group by method is activated. [#3254](https://github.com/ClickHouse/ClickHouse/pull/3254) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement + +* Added CA certificates to clickhouse-server docker image. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). + + ### ClickHouse release v20.1.10.70, 2020-04-17 #### Bug Fix From a6f118465bd1fb8add44735c20edbbbf9ac49472 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 29 May 2020 16:19:31 +0300 Subject: [PATCH 072/150] better --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 14638636d2b..d825abcf226 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -738,8 +738,8 @@ * Fixed improper shutdown of Distributed storage. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). * Fixed numeric overflow in `simpleLinearRegression` over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). * Fixed removing metadata directory when attach database fails. [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)). -* Added a check of number and type of arguments when creating BloomFilter index [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623). [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed the issue when a query with ARRAY JOIN, ORDER BY and LIMIT may return incomplete result. This fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). Author: [Vadim Plakhtinskiy](https://github.com/VadimPlh). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added a check of number and type of arguments when creating `BloomFilter` index [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623). [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed the issue when a query with `ARRAY JOIN`, `ORDER BY` and `LIMIT` may return incomplete result. This fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Prefer `fallback_to_stale_replicas` over `skip_unavailable_shards`. [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)). * Fixed wrong flattening of `Array(Tuple(...))` data types. This fixes [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259). [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fixed wrong behavior in `HashTable` that caused compilation error when trying to read HashMap from buffer. [#10386](https://github.com/ClickHouse/ClickHouse/pull/10386) ([palasonic1](https://github.com/palasonic1)). From cbd961ed8d672f7c87cc697df11d60f2d3f3cca3 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 29 May 2020 16:21:57 +0300 Subject: [PATCH 073/150] typo --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d825abcf226..f32bf8dde21 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -39,7 +39,7 @@ * Fixed `SELECT` of column `ALIAS` which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). * Implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). * Disable `GROUP BY` sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for `WITH ROLLUP/CUBE/TOTALS`. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). -* Fixed [#10263] (https://github.com/ClickHouse/ClickHouse/issues/10263). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). +* Fixed [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). * Added failing tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Added backward compatibility for create bloom filter index. [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). From 07205519b5b2649665700ebb13daabeb87bd7181 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 16:31:27 +0300 Subject: [PATCH 074/150] Addition to #11184 --- docker/test/stateful/Dockerfile | 2 -- docker/test/stateful_with_coverage/run.sh | 2 -- docker/test/stateless/Dockerfile | 2 -- docker/test/stateless_with_coverage/run.sh | 2 -- docker/test/stress/Dockerfile | 1 - tests/config/metric_log.xml | 8 -------- 6 files changed, 17 deletions(-) delete mode 100644 tests/config/metric_log.xml diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index e51efadf653..ace9e0d46a5 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -24,8 +24,6 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/; \ diff --git a/docker/test/stateful_with_coverage/run.sh b/docker/test/stateful_with_coverage/run.sh index b946f5b187d..5530aadb4ca 100755 --- a/docker/test/stateful_with_coverage/run.sh +++ b/docker/test/stateful_with_coverage/run.sh @@ -59,9 +59,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/; \ diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 35a8a5a9d3d..ad64e2e9880 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -62,9 +62,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index 185dc95c783..12ed7a25b75 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -50,9 +50,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index a5aa3bbf004..66f5135a4a4 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -31,7 +31,6 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ dpkg -i package_folder/clickhouse-server_*.deb; \ dpkg -i package_folder/clickhouse-client_*.deb; \ dpkg -i package_folder/clickhouse-test_*.deb; \ - ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; \ echo "TSAN_OPTIONS='halt_on_error=1 history_size=7 ignore_noninstrumented_modules=1 verbosity=1'" >> /etc/environment; \ diff --git a/tests/config/metric_log.xml b/tests/config/metric_log.xml deleted file mode 100644 index 0ca9f162416..00000000000 --- a/tests/config/metric_log.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - system - metric_log
- 7500 - 1000 -
-
From 080b2ec891dbe8de1fc0ff7ddfe99d465e0e7438 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 17:04:16 +0300 Subject: [PATCH 075/150] Attempt to fix flacky test --- .../01108_restart_replicas_rename_deadlock.reference | 4 ++++ .../0_stateless/01108_restart_replicas_rename_deadlock.sh | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference index 337ab7fe2e3..5e89d51914c 100644 --- a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference +++ b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference @@ -1 +1,5 @@ +replica_01108_1_tmp +replica_01108_2_tmp +replica_01108_3_tmp +replica_01108_4_tmp 1180 40 diff --git a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh index 01c9ef0a277..d338c8692a8 100755 --- a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh +++ b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh @@ -63,6 +63,12 @@ timeout $TIMEOUT bash -c restart_thread_2 2> /dev/null & wait sleep 3 +for i in `seq 4`; do + $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA replica_01108_$i" >/dev/null 2>&1 + $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA replica_01108_${i}_tmp" >/dev/null 2>&1 +done + +$CLICKHOUSE_CLIENT -q "SHOW TABLES LIKE 'replica\\_01108\\_%'" $CLICKHOUSE_CLIENT -q "SELECT sum(n), count(n) FROM merge(currentDatabase(), '^replica_01108_') GROUP BY position(_table, 'tmp')" From e940229e7124d4f0e9c7fcd1691480e12e0f1c6c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 17:09:28 +0300 Subject: [PATCH 076/150] Attempt to fix flacky test --- .../01108_restart_replicas_rename_deadlock.reference | 8 ++++---- .../0_stateless/01108_restart_replicas_rename_deadlock.sh | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference index 5e89d51914c..89a55127234 100644 --- a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference +++ b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.reference @@ -1,5 +1,5 @@ -replica_01108_1_tmp -replica_01108_2_tmp -replica_01108_3_tmp -replica_01108_4_tmp +replica_01108_1 +replica_01108_2 +replica_01108_3 +replica_01108_4 1180 40 diff --git a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh index d338c8692a8..65b738aed8e 100755 --- a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh +++ b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock.sh @@ -53,7 +53,7 @@ export -f rename_thread_2; export -f restart_thread_1; export -f restart_thread_2; -TIMEOUT=30 +TIMEOUT=10 timeout $TIMEOUT bash -c rename_thread_1 2> /dev/null & timeout $TIMEOUT bash -c rename_thread_2 2> /dev/null & @@ -68,7 +68,7 @@ for i in `seq 4`; do $CLICKHOUSE_CLIENT -q "SYSTEM SYNC REPLICA replica_01108_${i}_tmp" >/dev/null 2>&1 done -$CLICKHOUSE_CLIENT -q "SHOW TABLES LIKE 'replica\\_01108\\_%'" +$CLICKHOUSE_CLIENT -q "SELECT replaceOne(name, '_tmp', '') FROM system.tables WHERE database = currentDatabase() AND match(name, '^replica_01108_')" $CLICKHOUSE_CLIENT -q "SELECT sum(n), count(n) FROM merge(currentDatabase(), '^replica_01108_') GROUP BY position(_table, 'tmp')" From f7e0e87c7df29bbb8e9def7c0c2bfb6c459465cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 May 2020 19:35:09 +0300 Subject: [PATCH 077/150] Try fix tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Processors/Transforms/ConvertingTransform.cpp | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fb3188545f1..56532e532d1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -450,7 +450,7 @@ BlockIO InterpreterSelectQuery::execute() { res.pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, result_header, ConvertingTransform::MatchColumnsMode::Position); + return std::make_shared(header, result_header, ConvertingTransform::MatchColumnsMode::Name); }); } diff --git a/src/Processors/Transforms/ConvertingTransform.cpp b/src/Processors/Transforms/ConvertingTransform.cpp index c5bf5ad4d70..a0e9626b302 100644 --- a/src/Processors/Transforms/ConvertingTransform.cpp +++ b/src/Processors/Transforms/ConvertingTransform.cpp @@ -59,7 +59,11 @@ ConvertingTransform::ConvertingTransform( break; case MatchColumnsMode::Name: - if (source.has(res_elem.name)) + /// It may seem strange, but sometimes block may have columns with the same name. + /// For this specific case, try to get column from the same position if it has correct name first. + if (result_col_num < source.columns() && source.getByPosition(result_col_num).name == res_elem.name) + conversion[result_col_num] = result_col_num; + else if (source.has(res_elem.name)) conversion[result_col_num] = source.getPositionByName(res_elem.name); else throw Exception("Cannot find column " + backQuoteIfNeed(res_elem.name) + " in source stream", From a5f60e9cce5d27190b997e83866c78b736a35214 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 29 May 2020 20:11:08 +0300 Subject: [PATCH 078/150] trigger sync check From 36b1a3792795b60e49ed8be6fbe121b3c6c79d39 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 May 2020 21:23:11 +0300 Subject: [PATCH 079/150] Revert setting. --- src/Core/Settings.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 142e0872d72..e07f624408f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -437,6 +437,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \ M(SettingBool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \ M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \ + M(SettingBool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) From 9d39fc6dcbc49b8a6b187a30fb7a5354229efc76 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 21:23:45 +0300 Subject: [PATCH 080/150] Fix bug with Throttler and query speed estimation --- src/DataStreams/ExecutionSpeedLimits.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataStreams/ExecutionSpeedLimits.cpp b/src/DataStreams/ExecutionSpeedLimits.cpp index 28a8cd94994..e1927d187f5 100644 --- a/src/DataStreams/ExecutionSpeedLimits.cpp +++ b/src/DataStreams/ExecutionSpeedLimits.cpp @@ -52,7 +52,7 @@ void ExecutionSpeedLimits::throttle( UInt64 throttler_sleep_microseconds = CurrentThread::getProfileEvents()[ProfileEvents::ThrottlerSleepMicroseconds]; double elapsed_seconds = 0; - if (throttler_sleep_microseconds > total_elapsed_microseconds) + if (total_elapsed_microseconds > throttler_sleep_microseconds) elapsed_seconds = static_cast(total_elapsed_microseconds - throttler_sleep_microseconds) / 1000000.0; if (elapsed_seconds > 0) From 0eb2f4dee91117eef0ff4d3cde0f2a939a33b971 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 21:51:51 +0300 Subject: [PATCH 081/150] Change default value of "timeout_before_checking_execution_speed" to make it reasonable --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 142e0872d72..b1b0c89f66f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -310,7 +310,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, max_execution_speed, 0, "Maximum number of execution rows per second.", 0) \ M(SettingUInt64, min_execution_speed_bytes, 0, "Minimum number of execution bytes per second.", 0) \ M(SettingUInt64, max_execution_speed_bytes, 0, "Maximum number of execution bytes per second.", 0) \ - M(SettingSeconds, timeout_before_checking_execution_speed, 0, "Check that the speed is not too low after the specified time has elapsed.", 0) \ + M(SettingSeconds, timeout_before_checking_execution_speed, 10, "Check that the speed is not too low after the specified time has elapsed.", 0) \ \ M(SettingUInt64, max_columns_to_read, 0, "", 0) \ M(SettingUInt64, max_temporary_columns, 0, "", 0) \ From 6ec9287925d9d42ba419b6830ac52f1aa81d44f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 22:37:23 +0300 Subject: [PATCH 082/150] Added a test --- .../01287_max_execution_speed.reference | 10 +++++ .../0_stateless/01287_max_execution_speed.sql | 40 +++++++++++++++++++ 2 files changed, 50 insertions(+) create mode 100644 tests/queries/0_stateless/01287_max_execution_speed.reference create mode 100644 tests/queries/0_stateless/01287_max_execution_speed.sql diff --git a/tests/queries/0_stateless/01287_max_execution_speed.reference b/tests/queries/0_stateless/01287_max_execution_speed.reference new file mode 100644 index 00000000000..5a3fde429cf --- /dev/null +++ b/tests/queries/0_stateless/01287_max_execution_speed.reference @@ -0,0 +1,10 @@ +Ok (1) +Ok (2) +2000000 +1 +Ok (3) +2000000 +1 +Ok (4) +1000000 +Ok (5) diff --git a/tests/queries/0_stateless/01287_max_execution_speed.sql b/tests/queries/0_stateless/01287_max_execution_speed.sql new file mode 100644 index 00000000000..921a5f77dd5 --- /dev/null +++ b/tests/queries/0_stateless/01287_max_execution_speed.sql @@ -0,0 +1,40 @@ +SET min_execution_speed = 100000000000, timeout_before_checking_execution_speed = 0.1; +SELECT count() FROM system.numbers; -- { serverError 160 } +SELECT 'Ok (1)'; +SET min_execution_speed = 0; + +SET min_execution_speed_bytes = 800000000000, timeout_before_checking_execution_speed = 0.1; +SELECT count() FROM system.numbers; -- { serverError 160 } +SELECT 'Ok (2)'; +SET min_execution_speed_bytes = 0; + +SET max_execution_speed = 1000000; +SET max_block_size = 100; + +CREATE TEMPORARY TABLE times (t DateTime); + +INSERT INTO times SELECT now(); +SELECT count() FROM numbers(2000000); +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; +SELECT 'Ok (3)'; +SET max_execution_speed = 0; + +SET max_execution_speed_bytes = 8000000; +TRUNCATE TABLE times; + +INSERT INTO times SELECT now(); +SELECT count() FROM numbers(2000000); +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; +SELECT 'Ok (4)'; +SET max_execution_speed_bytes = 0; + +-- Note that 'min_execution_speed' does not count sleeping due to throttling +-- with 'max_execution_speed' and similar limits like 'priority' and 'max_network_bandwidth' +SET max_execution_speed = 1000000, min_execution_speed = 2000000; +-- And this query will work despite the fact that the above settings look contradictory. +SELECT count() FROM numbers(1000000); +SELECT 'Ok (5)'; From 1732888d98606e9fa6bb94068038c28082a082f0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 22:44:24 +0300 Subject: [PATCH 083/150] Added a test for network throttler --- .../01288_shard_max_network_bandwidth.reference | 2 ++ .../01288_shard_max_network_bandwidth.sql | 15 +++++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/01288_shard_max_network_bandwidth.reference create mode 100644 tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql diff --git a/tests/queries/0_stateless/01288_shard_max_network_bandwidth.reference b/tests/queries/0_stateless/01288_shard_max_network_bandwidth.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/tests/queries/0_stateless/01288_shard_max_network_bandwidth.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql b/tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql new file mode 100644 index 00000000000..09c043784bb --- /dev/null +++ b/tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql @@ -0,0 +1,15 @@ +-- Limit to 10 MB/sec +SET max_network_bandwidth = 10000000; + +-- Lower max_block_size, so we can start throttling sooner. Otherwise query will be executed too quickly. +SET max_block_size = 100; + +CREATE TEMPORARY TABLE times (t DateTime); + +-- rand64 is uncompressable data. Each number will take 8 bytes of bandwidth. +-- This query should execute in no less than 1.6 seconds if throttled. +INSERT INTO times SELECT now(); +SELECT sum(ignore(*)) FROM (SELECT rand64() FROM remote('127.0.0.{2,3}', numbers(2000000))); +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; From fb5e1a8e8d82503a134c1a45fb680ab4d9df264b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 22:54:59 +0300 Subject: [PATCH 084/150] Whitespaces --- src/DataStreams/ExecutionSpeedLimits.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataStreams/ExecutionSpeedLimits.cpp b/src/DataStreams/ExecutionSpeedLimits.cpp index e1927d187f5..6eb138ec753 100644 --- a/src/DataStreams/ExecutionSpeedLimits.cpp +++ b/src/DataStreams/ExecutionSpeedLimits.cpp @@ -45,8 +45,8 @@ void ExecutionSpeedLimits::throttle( { if ((min_execution_rps != 0 || max_execution_rps != 0 || min_execution_bps != 0 || max_execution_bps != 0 - || (total_rows_to_read != 0 && timeout_before_checking_execution_speed != 0)) && - (static_cast(total_elapsed_microseconds) > timeout_before_checking_execution_speed.totalMicroseconds())) + || (total_rows_to_read != 0 && timeout_before_checking_execution_speed != 0)) + && (static_cast(total_elapsed_microseconds) > timeout_before_checking_execution_speed.totalMicroseconds())) { /// Do not count sleeps in throttlers UInt64 throttler_sleep_microseconds = CurrentThread::getProfileEvents()[ProfileEvents::ThrottlerSleepMicroseconds]; From 3d6057976ed341d4d989d087d4351440d1c746a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 23:00:51 +0300 Subject: [PATCH 085/150] Fix bad formatting in clickhouse-client --- programs/client/Client.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index afc8f9a72b1..ff119d89ce6 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1583,7 +1583,7 @@ private: if (std::string::npos != embedded_stack_trace_pos && !config().getBool("stacktrace", false)) text.resize(embedded_stack_trace_pos); - std::cerr << "Received exception from server (version " << server_version << "):" << std::endl + std::cerr << "\nReceived exception from server (version " << server_version << "):" << std::endl << "Code: " << e.code() << ". " << text << std::endl; } From cf91793d90e6b34adf10b89e5162aed3573f0c22 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 23:05:41 +0300 Subject: [PATCH 086/150] Added a test from @den-crane --- ...in_execution_speed_not_too_early.reference | 0 ...1289_min_execution_speed_not_too_early.sql | 19 +++++++++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/01289_min_execution_speed_not_too_early.reference create mode 100644 tests/queries/0_stateless/01289_min_execution_speed_not_too_early.sql diff --git a/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.reference b/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.sql b/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.sql new file mode 100644 index 00000000000..e3a18d0c515 --- /dev/null +++ b/tests/queries/0_stateless/01289_min_execution_speed_not_too_early.sql @@ -0,0 +1,19 @@ +DROP TABLE IF EXISTS ES; + +create table ES(A String) Engine=MergeTree order by tuple(); +insert into ES select toString(number) from numbers(10000000); + +SET max_execution_time = 100, max_execution_speed = 1000000; +SET max_threads = 1; +SET max_block_size = 1000000; + +-- Exception about execution speed is not thrown from these queries. +SELECT * FROM ES LIMIT 1 format Null; +SELECT * FROM ES LIMIT 10 format Null; +SELECT * FROM ES LIMIT 100 format Null; +SELECT * FROM ES LIMIT 1000 format Null; +SELECT * FROM ES LIMIT 10000 format Null; +SELECT * FROM ES LIMIT 100000 format Null; +SELECT * FROM ES LIMIT 1000000 format Null; + +DROP TABLE ES; From 69afc13d55d3db72c4a85e4b683e5994e0500213 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 29 May 2020 23:09:01 +0300 Subject: [PATCH 087/150] Fix build scripts related to protobuf and gRPC for Mac. --- cmake/protobuf_generate_cpp.cmake | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cmake/protobuf_generate_cpp.cmake b/cmake/protobuf_generate_cpp.cmake index 7ee9e8d7c81..cc2502e5eeb 100644 --- a/cmake/protobuf_generate_cpp.cmake +++ b/cmake/protobuf_generate_cpp.cmake @@ -55,6 +55,7 @@ function(protobuf_generate_cpp_impl SRCS HDRS MODES OUTPUT_FILE_EXTS PLUGIN) endif() set (intermediate_dir ${CMAKE_CURRENT_BINARY_DIR}/intermediate) + file (MAKE_DIRECTORY ${intermediate_dir}) set (protoc_args) foreach (mode ${MODES}) @@ -112,7 +113,7 @@ if (PROTOBUF_GENERATE_CPP_SCRIPT_MODE) set (intermediate_dir ${DIR}/intermediate) set (intermediate_output "${intermediate_dir}/${FILENAME}") - if (COMPILER_ID STREQUAL "Clang") + if (COMPILER_ID MATCHES "Clang") set (pragma_push "#pragma clang diagnostic push\n") set (pragma_pop "#pragma clang diagnostic pop\n") set (pragma_disable_warnings "#pragma clang diagnostic ignored \"-Weverything\"\n") From 9768743d09f15c1525a208ee2e307d97ea7fdf0f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 23:17:38 +0300 Subject: [PATCH 088/150] Whitespace --- src/DataStreams/ExecutionSpeedLimits.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataStreams/ExecutionSpeedLimits.cpp b/src/DataStreams/ExecutionSpeedLimits.cpp index 6eb138ec753..6cc1b9006bf 100644 --- a/src/DataStreams/ExecutionSpeedLimits.cpp +++ b/src/DataStreams/ExecutionSpeedLimits.cpp @@ -30,7 +30,8 @@ static void limitProgressingSpeed(size_t total_progress_size, size_t max_speed_i { UInt64 sleep_microseconds = desired_microseconds - total_elapsed_microseconds; - /// Never sleep more than one second (it should be enough to limit speed for a reasonable amount, and otherwise it's too easy to make query hang). + /// Never sleep more than one second (it should be enough to limit speed for a reasonable amount, + /// and otherwise it's too easy to make query hang). sleep_microseconds = std::min(UInt64(1000000), sleep_microseconds); sleepForMicroseconds(sleep_microseconds); From fe9bd0e9181948a7a645b6bae7c84d5715be383b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 23:37:14 +0300 Subject: [PATCH 089/150] Added a test for #5732 --- .../01290_max_execution_speed_distributed.reference | 4 ++++ .../01290_max_execution_speed_distributed.sql | 13 +++++++++++++ 2 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/01290_max_execution_speed_distributed.reference create mode 100644 tests/queries/0_stateless/01290_max_execution_speed_distributed.sql diff --git a/tests/queries/0_stateless/01290_max_execution_speed_distributed.reference b/tests/queries/0_stateless/01290_max_execution_speed_distributed.reference new file mode 100644 index 00000000000..48355e41fe4 --- /dev/null +++ b/tests/queries/0_stateless/01290_max_execution_speed_distributed.reference @@ -0,0 +1,4 @@ +2000000 +1 +1 +0 diff --git a/tests/queries/0_stateless/01290_max_execution_speed_distributed.sql b/tests/queries/0_stateless/01290_max_execution_speed_distributed.sql new file mode 100644 index 00000000000..8282390ca90 --- /dev/null +++ b/tests/queries/0_stateless/01290_max_execution_speed_distributed.sql @@ -0,0 +1,13 @@ +SET max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0.001, max_block_size = 100; + +CREATE TEMPORARY TABLE times (t DateTime); + +INSERT INTO times SELECT now(); +SELECT count('special query for 01290_max_execution_speed_distributed') FROM remote('127.0.0.{2,3}', numbers(1000000)); +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; + +-- Check that the query was also throttled on "remote" servers. +SYSTEM FLUSH LOGS; +SELECT DISTINCT query_duration_ms >= 500 FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%special query for 01290_max_execution_speed_distributed%' AND type = 2; From e4a329a47fb75fcfb8548c2a0b8073a5d62483a1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 23:37:37 +0300 Subject: [PATCH 090/150] Checking max execution speed on all nodes #5732 --- src/Interpreters/InterpreterSelectQuery.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 4f717eda706..5fa1bfcdc6a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1498,16 +1498,19 @@ void InterpreterSelectQuery::executeFetchColumns( * But limits on data size to read and maximum execution time are reasonable to check both on initiator and * additionally on each remote server, because these limits are checked per block of data processed, * and remote servers may process way more blocks of data than are received by initiator. + * + * The limits to throttle maximum execution speed is also checked on all servers. */ if (options.to_stage == QueryProcessingStage::Complete) { limits.speed_limits.min_execution_rps = settings.min_execution_speed; - limits.speed_limits.max_execution_rps = settings.max_execution_speed; limits.speed_limits.min_execution_bps = settings.min_execution_speed_bytes; - limits.speed_limits.max_execution_bps = settings.max_execution_speed_bytes; - limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; } + limits.speed_limits.max_execution_rps = settings.max_execution_speed; + limits.speed_limits.max_execution_bps = settings.max_execution_speed_bytes; + limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; + auto quota = context->getQuota(); for (auto & stream : streams) From 2ba268c80ad713e03a1e5bedfb8350c68a958ded Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 23:45:47 +0300 Subject: [PATCH 091/150] Added a test for #6228 --- ...56_max_execution_speed_sample_merge.reference | 4 ++++ .../00156_max_execution_speed_sample_merge.sql | 16 ++++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 tests/queries/1_stateful/00156_max_execution_speed_sample_merge.reference create mode 100644 tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql diff --git a/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.reference b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.reference new file mode 100644 index 00000000000..53bb58224b9 --- /dev/null +++ b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.reference @@ -0,0 +1,4 @@ +4392010 +1 +4392010 +1 diff --git a/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql new file mode 100644 index 00000000000..37e91296f14 --- /dev/null +++ b/tests/queries/1_stateful/00156_max_execution_speed_sample_merge.sql @@ -0,0 +1,16 @@ +SET max_execution_speed = 4000000, timeout_before_checking_execution_speed = 0.001; + +CREATE TEMPORARY TABLE times (t DateTime); + +INSERT INTO times SELECT now(); +SELECT count() FROM test.hits SAMPLE 1 / 2; +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; +TRUNCATE TABLE times; + +INSERT INTO times SELECT now(); +SELECT count() FROM merge(test, '^hits$') SAMPLE 1 / 2; +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; From ff12b9c54677214e90e2bd32b7724350ec4a1d93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 00:00:48 +0300 Subject: [PATCH 092/150] Fix bad sh script in Dockerfile --- docker/test/stateful/Dockerfile | 4 ++-- docker/test/stateless/Dockerfile | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index e51efadf653..3aff49bf5a1 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -32,8 +32,8 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; \ - if [ -n $USE_DATABASE_ATOMIC ] && [ $USE_DATABASE_ATOMIC -eq 1 ]; then ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/; fi; \ - if [ -n $USE_DATABASE_ATOMIC ] && [ $USE_DATABASE_ATOMIC -eq 1 ]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/; fi; \ + if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/; fi; \ + if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/; fi; \ echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-8/bin/llvm-symbolizer" >> /etc/environment; \ echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 35a8a5a9d3d..41a53f8a3f5 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -78,9 +78,9 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/; \ - if [ -n $USE_POLYMORPHIC_PARTS ] && [ $USE_POLYMORPHIC_PARTS -eq 1 ]; then ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/; fi; \ - if [ -n $USE_DATABASE_ATOMIC ] && [ $USE_DATABASE_ATOMIC -eq 1 ]; then ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/; fi; \ - if [ -n $USE_DATABASE_ATOMIC ] && [ $USE_DATABASE_ATOMIC -eq 1 ]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/; fi; \ + if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/; fi; \ + if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/; fi; \ + if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/; fi; \ ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml; \ service zookeeper start; sleep 5; \ service clickhouse-server start && sleep 5 && clickhouse-test --testname --shard --zookeeper $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt From 8c8821475cb25d221780f2fc1bda240278fe860a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 00:32:35 +0300 Subject: [PATCH 093/150] Fix issue #11286; add a test --- src/Storages/MergeTree/KeyCondition.cpp | 10 ++- ...01290_empty_array_index_analysis.reference | 50 ++++++++++++++ .../01290_empty_array_index_analysis.sql | 66 +++++++++++++++++++ 3 files changed, 124 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01290_empty_array_index_analysis.reference create mode 100644 tests/queries/0_stateless/01290_empty_array_index_analysis.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 6ae22885dfd..ee381709dd4 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -181,8 +181,11 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "empty", - [] (RPNElement & out, const Field &) + [] (RPNElement & out, const Field & value) { + if (value.getType() != Field::Types::String) + return false; + out.function = RPNElement::FUNCTION_IN_RANGE; out.range = Range(""); return true; @@ -190,8 +193,11 @@ const KeyCondition::AtomMap KeyCondition::atom_map }, { "notEmpty", - [] (RPNElement & out, const Field &) + [] (RPNElement & out, const Field & value) { + if (value.getType() != Field::Types::String) + return false; + out.function = RPNElement::FUNCTION_NOT_IN_RANGE; out.range = Range(""); return true; diff --git a/tests/queries/0_stateless/01290_empty_array_index_analysis.reference b/tests/queries/0_stateless/01290_empty_array_index_analysis.reference new file mode 100644 index 00000000000..5037a64c0f0 --- /dev/null +++ b/tests/queries/0_stateless/01290_empty_array_index_analysis.reference @@ -0,0 +1,50 @@ +--- notEmpty + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- empty + [] 1 +--- = [] + [] 1 +--- != [] + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- > [] + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- < [] +--- >= [] + [] 1 + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- <= [] + [] 1 +--- +--- notEmpty + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- empty + [] 1 +--- = [] + [] 1 +--- != [] + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- > [] + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- < [] +--- >= [] + [] 1 + ['a'] 2 + ['a','b','c'] 3 + ['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa'] 4 +--- <= [] + [] 1 +--- diff --git a/tests/queries/0_stateless/01290_empty_array_index_analysis.sql b/tests/queries/0_stateless/01290_empty_array_index_analysis.sql new file mode 100644 index 00000000000..b1b6067945d --- /dev/null +++ b/tests/queries/0_stateless/01290_empty_array_index_analysis.sql @@ -0,0 +1,66 @@ +drop table if exists count_lc_test; + +CREATE TABLE count_lc_test +( + `s` LowCardinality(String), + `arr` Array(LowCardinality(String)), + `num` UInt64 +) +ENGINE = MergeTree +ORDER BY (s, arr); + +INSERT INTO count_lc_test(num, arr) VALUES (1,[]),(2,['a']),(3,['a','b','c']),(4,['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa']); + +SELECT '--- notEmpty'; +select * from count_lc_test where notEmpty(arr); +SELECT '--- empty'; +select * from count_lc_test where empty(arr); +SELECT '--- = []'; +select * from count_lc_test where arr = []; +SELECT '--- != []'; +select * from count_lc_test where arr != []; +SELECT '--- > []'; +select * from count_lc_test where arr > []; +SELECT '--- < []'; +select * from count_lc_test where arr < []; +SELECT '--- >= []'; +select * from count_lc_test where arr >= []; +SELECT '--- <= []'; +select * from count_lc_test where arr <= []; +SELECT '---'; + +DROP TABLE count_lc_test; + + +drop table if exists count_lc_test; + +CREATE TABLE count_lc_test +( + `s` LowCardinality(String), + `arr` Array(String), + `num` UInt64 +) +ENGINE = MergeTree +ORDER BY (s, arr); + +INSERT INTO count_lc_test(num, arr) VALUES (1,[]),(2,['a']),(3,['a','b','c']),(4,['aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa']); + +SELECT '--- notEmpty'; +select * from count_lc_test where notEmpty(arr); +SELECT '--- empty'; +select * from count_lc_test where empty(arr); +SELECT '--- = []'; +select * from count_lc_test where arr = []; +SELECT '--- != []'; +select * from count_lc_test where arr != []; +SELECT '--- > []'; +select * from count_lc_test where arr > []; +SELECT '--- < []'; +select * from count_lc_test where arr < []; +SELECT '--- >= []'; +select * from count_lc_test where arr >= []; +SELECT '--- <= []'; +select * from count_lc_test where arr <= []; +SELECT '---'; + +DROP TABLE count_lc_test; From b79020d0a81796e38ce3ac3dfa10eab4c8869430 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 00:41:33 +0300 Subject: [PATCH 094/150] Emit a warning if server was build in debug or with sanitizers --- programs/server/Server.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ce1d35e65d4..29096327a71 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -236,6 +236,14 @@ int Server::main(const std::vector & /*args*/) if (ThreadFuzzer::instance().isEffective()) LOG_WARNING(log, "ThreadFuzzer is enabled. Application will run slowly and unstable."); +#if !defined(NDEBUG) || !defined(__OPTIMIZE__) + LOG_WARNING(log, "Server was built in debug mode. It will work slowly."); +#endif + +#if defined(ADDRESS_SANITIZER) || defined(THREAD_SANITIZER) || defined(MEMORY_SANITIZER) + LOG_WARNING(log, "Server was built with sanitizer. It will work slowly."); +#endif + /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases... */ From b78c48310bb7bb3267cdc70acbc8ce2135f8fcc1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 01:37:44 +0300 Subject: [PATCH 095/150] Whitespaces --- src/IO/ReadHelpers.cpp | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 0797e63cdd9..9d1d27611b5 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -650,7 +650,6 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & ++next_pos; }(); - appendToStringOrVector(s, buf, next_pos); buf.position() = next_pos; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 21768644940..287bf916c19 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -775,6 +775,7 @@ void IMergeTreeDataPart::remove() const } } + String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) const { /// Do not allow underscores in the prefix because they are used as separators. From ffdec23d08d011b9a0c38adfc1ba380495b30346 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 01:43:25 +0300 Subject: [PATCH 096/150] Remove wrong code #8441 --- src/Server/HTTPHandler.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index e5c784fd409..82068496159 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -404,8 +404,6 @@ void HTTPHandler::processQuery( used_output.out_maybe_delayed_and_compressed = used_output.out_maybe_compressed; } - std::unique_ptr in_post_raw = std::make_unique(istr); - /// Request body can be compressed using algorithm specified in the Content-Encoding header. String http_request_compression_method_str = request.get("Content-Encoding", ""); std::unique_ptr in_post = wrapReadBufferWithCompressionMethod( From 9a3d449da03fce35ffd857bdb9216fe658167f6d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 30 May 2020 10:36:00 +0300 Subject: [PATCH 097/150] Fix copier. --- programs/copier/ClusterCopier.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index d6a2fe103d1..600f110e857 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1774,7 +1774,7 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi Context local_context = context; local_context.setSettings(task_cluster->settings_pull); - auto result = InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows(); + auto result = InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows(); if (result != 0) LOG_DEBUG(log, "Partition {} piece number {} is PRESENT on shard {}", partition_quoted_name, std::to_string(current_piece_number), task_shard.getDescription()); else From 96568566eed8e4560f749b6fcc7c22d498a9975e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 30 May 2020 10:50:45 +0300 Subject: [PATCH 098/150] Fix copier. --- programs/copier/ClusterCopier.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 600f110e857..77ec1421391 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1403,7 +1403,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( BlockIO io_select = InterpreterFactory::get(query_select_ast, context_select)->execute(); BlockIO io_insert = InterpreterFactory::get(query_insert_ast, context_insert)->execute(); - input = io_select.in; + input = io_select.getInputStream(); output = io_insert.out; } From 134f9f2b5bc8ca3cf67d87a52b9959b0f94aa9be Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 30 May 2020 11:07:12 +0300 Subject: [PATCH 099/150] Fix copier. --- programs/copier/ClusterCopier.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 77ec1421391..227ce9eb715 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1293,7 +1293,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( local_context.setSettings(task_cluster->settings_pull); local_context.setSetting("skip_unavailable_shards", true); - Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().in); + Block block = getBlockWithAllStreamData(InterpreterFactory::get(query_select_ast, local_context)->execute().getInputStream()); count = (block) ? block.safeGetByPosition(0).column->getUInt(0) : 0; } @@ -1735,7 +1735,7 @@ const auto & settings = context.getSettingsRef(); Context local_context = context; local_context.setSettings(task_cluster->settings_pull); - return InterpreterFactory::get(query_ast, local_context)->execute().in->read().rows() != 0; + return InterpreterFactory::get(query_ast, local_context)->execute().getInputStream()->read().rows() != 0; } bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTimeouts & timeouts, From c43976b6c581d699c5caf9632f53e77e84600399 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 30 May 2020 11:31:07 +0300 Subject: [PATCH 100/150] Update test. --- tests/integration/test_ttl_move/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index c6453beea6a..fc5c6725148 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -864,6 +864,8 @@ def test_double_move_while_select(started_cluster, name, positive): thread = threading.Thread(target=long_select) thread.start() + time.sleep(1) + node1.query("ALTER TABLE {name} MOVE PART '{part}' TO DISK 'jbod1'".format(name=name, part=parts[0])) # Fill jbod1 to force ClickHouse to make move of partition 1 to external. From b84c8fcbd84067b9e1464ba102668b832f8c1ee1 Mon Sep 17 00:00:00 2001 From: Alexey Ilyukhov Date: Sat, 30 May 2020 18:26:19 +0300 Subject: [PATCH 101/150] Add allow_experimental_geo_types; add test --- src/Core/Settings.h | 2 + src/DataTypes/DataTypeCustomGeo.cpp | 79 +++++++++++++------ src/Interpreters/InterpreterCreateQuery.cpp | 17 +++- .../0_stateless/01291_geo_types.reference | 1 + tests/queries/0_stateless/01291_geo_types.sql | 9 +++ 5 files changed, 84 insertions(+), 24 deletions(-) create mode 100644 tests/queries/0_stateless/01291_geo_types.reference create mode 100644 tests/queries/0_stateless/01291_geo_types.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 61fcf658ba8..36d93f5814a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -416,6 +416,8 @@ struct Settings : public SettingsCollection M(SettingBool, allow_nondeterministic_mutations, false, "Allow non-deterministic functions in ALTER UPDATE/ALTER DELETE statements", 0) \ M(SettingSeconds, lock_acquire_timeout, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "How long locking request should wait before failing", 0) \ \ + M(SettingBool, allow_experimental_geo_types, false, "Allow geo data types such as Point, Ring, Polygon, MultiPolygon", 0) \ + \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13", 0) \ diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index d3d808a1b50..2f70e9f732a 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -19,62 +19,89 @@ private: DataTypePtr tuple; public: - DataTypeCustomPointSerialization() : tuple(std::make_unique( - DataTypes({std::make_unique(), std::make_unique()}))) + DataTypeCustomPointSerialization() {} void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - tuple->serializeAsText(column, row_num, ostr, settings); + nestedDataType()->serializeAsText(column, row_num, ostr, settings); } void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - tuple->deserializeAsWholeText(column, istr, settings); + nestedDataType()->deserializeAsWholeText(column, istr, settings); + } + + static DataTypePtr nestedDataType() { + static auto dataType = DataTypePtr(std::make_unique( + DataTypes({std::make_unique(), std::make_unique()}))); + return dataType; + } +}; + +class DataTypeCustomRingSerialization : public DataTypeCustomSimpleTextSerialization +{ +public: + DataTypeCustomRingSerialization() + {} + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override + { + nestedDataType()->serializeAsText(column, row_num, ostr, settings); + } + + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + nestedDataType()->deserializeAsWholeText(column, istr, settings); + } + + static DataTypePtr nestedDataType() { + static auto dataType = DataTypePtr(std::make_unique(DataTypeCustomPointSerialization::nestedDataType())); + return dataType; } }; class DataTypeCustomPolygonSerialization : public DataTypeCustomSimpleTextSerialization { -private: - DataTypePtr array; - public: - DataTypeCustomPolygonSerialization() : array(std::make_unique(std::make_unique(std::make_unique( - DataTypes({std::make_unique(), std::make_unique()}))))) + DataTypeCustomPolygonSerialization() {} void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - array->serializeAsText(column, row_num, ostr, settings); + nestedDataType()->serializeAsText(column, row_num, ostr, settings); } void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - array->deserializeAsWholeText(column, istr, settings); + nestedDataType()->deserializeAsWholeText(column, istr, settings); + } + + static DataTypePtr nestedDataType() { + static auto dataType = DataTypePtr(std::make_unique(DataTypeCustomRingSerialization::nestedDataType())); + return dataType; } }; class DataTypeCustomMultiPolygonSerialization : public DataTypeCustomSimpleTextSerialization { -private: - DataTypePtr array; - public: - DataTypeCustomMultiPolygonSerialization() : array( - std::make_unique(std::make_unique( - std::make_unique(std::make_unique( - DataTypes({std::make_unique(), std::make_unique()})))))) + DataTypeCustomMultiPolygonSerialization() {} void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { - array->serializeAsText(column, row_num, ostr, settings); + nestedDataType()->serializeAsText(column, row_num, ostr, settings); } void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override { - array->deserializeAsWholeText(column, istr, settings); + nestedDataType()->deserializeAsWholeText(column, istr, settings); + } + + static DataTypePtr nestedDataType() { + static auto dataType = DataTypePtr(std::make_unique(DataTypeCustomPolygonSerialization::nestedDataType())); + return dataType; } }; @@ -89,12 +116,18 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory) std::make_unique(std::make_unique("Point"), std::make_unique())); }); - // Custom type for polygon with holes stored as Array(Array(Point)) - // Each element of outer array represents a simple polygon without holes stored as array of points + // Custom type for simple polygon without holes stored as Array(Point) + factory.registerSimpleDataTypeCustom("Ring", [] + { + return std::make_pair(DataTypeFactory::instance().get("Array(Point)"), + std::make_unique(std::make_unique("Ring"), std::make_unique())); + }); + + // Custom type for polygon with holes stored as Array(Ring) // First element of outer array is outer shape of polygon and all the following are holes factory.registerSimpleDataTypeCustom("Polygon", [] { - return std::make_pair(DataTypeFactory::instance().get("Array(Array(Point))"), + return std::make_pair(DataTypeFactory::instance().get("Array(Ring)"), std::make_unique(std::make_unique("Polygon"), std::make_unique())); }); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 43a0a7f608e..b9f11476edc 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -70,6 +70,7 @@ namespace ErrorCodes extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE; extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY; extern const int DICTIONARY_ALREADY_EXISTS; + extern const int ILLEGAL_COLUMN; } @@ -381,7 +382,6 @@ ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const A return res; } - InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(ASTCreateQuery & create) const { TableProperties properties; @@ -471,6 +471,21 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat } } } + + if (!create.attach && !context.getSettingsRef().allow_experimental_geo_types) + { + for (const auto & name_and_type_pair : properties.columns.getAllPhysical()) + { + const auto& type = name_and_type_pair.type->getName(); + if (type == "MultiPolygon" || type == "Polygon" || type == "Ring" || type == "Point") + { + String message = "Cannot create table with column '" + name_and_type_pair.name + "' which type is '" + + type + "' because experimental geo types are not allowed. " + + "Set setting allow_experimental_geo_types = 1 in order to allow it."; + throw Exception(message, ErrorCodes::ILLEGAL_COLUMN); + } + } + } } void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const diff --git a/tests/queries/0_stateless/01291_geo_types.reference b/tests/queries/0_stateless/01291_geo_types.reference new file mode 100644 index 00000000000..baf7b25a231 --- /dev/null +++ b/tests/queries/0_stateless/01291_geo_types.reference @@ -0,0 +1 @@ +(0,0) [(0,0),(10,0),(10,10),(0,10)] [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] diff --git a/tests/queries/0_stateless/01291_geo_types.sql b/tests/queries/0_stateless/01291_geo_types.sql new file mode 100644 index 00000000000..0d923f08ccd --- /dev/null +++ b/tests/queries/0_stateless/01291_geo_types.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS geo; + +SET allow_experimental_geo_types = 1; + +CREATE TABLE geo (a Point, b Ring, c Polygon, d MultiPolygon) ENGINE=Memory(); + +INSERT INTO geo VALUES((0, 0), [(0, 0), (10, 0), (10, 10), (0, 10)], [[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]], [[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]); + +SELECT * from geo; From 3b78b010e5baa7d6596c7f670ff0ee40096ae634 Mon Sep 17 00:00:00 2001 From: Alexey Ilyukhov Date: Sat, 30 May 2020 18:30:09 +0300 Subject: [PATCH 102/150] Add empty line --- src/Interpreters/InterpreterCreateQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b9f11476edc..39c0f779ea2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -382,6 +382,7 @@ ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const A return res; } + InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(ASTCreateQuery & create) const { TableProperties properties; From 808351048f89d75f59ed23f743de3e7356e67415 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sat, 30 May 2020 19:44:51 +0400 Subject: [PATCH 103/150] Add libsasl2-dev and heimdal-multidev required by static system openldap libraries --- docker/packager/deb/Dockerfile | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index ae70bc8c594..786e6620eac 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -80,7 +80,9 @@ RUN apt-get --allow-unauthenticated update -y \ pigz \ moreutils \ libcctz-dev \ - libldap2-dev + libldap2-dev \ + libsasl2-dev \ + heimdal-multidev From 6468d7af8c6d54a7bcf0cfd4ad481d11446222d6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 19:20:19 +0300 Subject: [PATCH 104/150] Regenerate ya.make --- src/Dictionaries/ya.make | 2 +- src/Functions/ya.make | 9 ++++----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index e47b55d5254..e30a22a372c 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -24,8 +24,8 @@ SRCS( ComplexKeyCacheDictionary_generate3.cpp ComplexKeyCacheDictionary_setAttributeValue.cpp ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp - ComplexKeyHashedDictionary.cpp ComplexKeyDirectDictionary.cpp + ComplexKeyHashedDictionary.cpp DictionaryBlockInputStreamBase.cpp DictionaryFactory.cpp DictionarySourceFactory.cpp diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 0b4776853e9..bbbf13d360b 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -171,7 +171,6 @@ SRCS( FunctionsRound.cpp FunctionsStringArray.cpp FunctionsStringSimilarity.cpp - FunctionUnixTimestamp64.h GatherUtils/concat.cpp GatherUtils/createArraySink.cpp GatherUtils/createArraySource.cpp @@ -285,10 +284,10 @@ SRCS( rand64.cpp randConstant.cpp rand.cpp + randomFixedString.cpp randomPrintableASCII.cpp randomString.cpp randomStringUTF8.cpp - randomFixedString.cpp regexpQuoteMeta.cpp registerFunctionsArithmetic.cpp registerFunctionsComparison.cpp @@ -308,8 +307,8 @@ SRCS( registerFunctionsStringRegexp.cpp registerFunctionsStringSearch.cpp registerFunctionsTuple.cpp - registerFunctionsVisitParam.cpp registerFunctionsUnixTimestamp64.cpp + registerFunctionsVisitParam.cpp reinterpretAsFixedString.cpp reinterpretAsString.cpp reinterpretStringAs.cpp @@ -390,10 +389,10 @@ SRCS( toTime.cpp toTimeZone.cpp toTypeName.cpp - toValidUTF8.cpp toUnixTimestamp64Micro.cpp toUnixTimestamp64Milli.cpp toUnixTimestamp64Nano.cpp + toValidUTF8.cpp toYear.cpp toYYYYMM.cpp toYYYYMMDD.cpp @@ -424,8 +423,8 @@ SRCS( URL/fragment.cpp URL/path.cpp URL/pathFull.cpp - URL/protocol.cpp URL/port.cpp + URL/protocol.cpp URL/queryStringAndFragment.cpp URL/queryString.cpp URL/registerFunctionsURL.cpp From 326d2f9af8a7b42700c49ae488449cc54b39fe0a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 19:21:14 +0300 Subject: [PATCH 105/150] Fix uninitialized memory in conversion --- src/Functions/FunctionsConversion.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9d0b764e84b..431bd9b2797 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -145,6 +145,8 @@ struct ConvertImpl vec_to[i] = convertFromDecimal(vec_from[i], vec_from.getScale()); else if constexpr (IsDataTypeNumber && IsDataTypeDecimal) vec_to[i] = convertToDecimal(vec_from[i], vec_to.getScale()); + else + throw Exception("Unsupported data type in conversion function", ErrorCodes::NOT_IMPLEMENTED); } else vec_to[i] = static_cast(vec_from[i]); From 1b825baf21c9f8549d0b24a3fe8b2fe6e26390bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 19:31:36 +0300 Subject: [PATCH 106/150] Added a test --- .../01291_unsupported_conversion_from_decimal.reference | 0 .../01291_unsupported_conversion_from_decimal.sql | 5 +++++ 2 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.reference create mode 100644 tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.sql diff --git a/tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.reference b/tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.sql b/tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.sql new file mode 100644 index 00000000000..256c6424901 --- /dev/null +++ b/tests/queries/0_stateless/01291_unsupported_conversion_from_decimal.sql @@ -0,0 +1,5 @@ +SELECT toIntervalSecond(now64()); -- { serverError 70 } +SELECT CAST(now64() AS IntervalSecond); -- { serverError 70 } + +SELECT toIntervalSecond(now64()); -- { serverError 70 } +SELECT CAST(now64() AS IntervalSecond); -- { serverError 70 } From 41bfa8a275bce9856c58c970f42508736545147b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 19:35:59 +0300 Subject: [PATCH 107/150] Addition to prev. revision --- src/Functions/FunctionsConversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 431bd9b2797..879b885cf66 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -146,7 +146,7 @@ struct ConvertImpl else if constexpr (IsDataTypeNumber && IsDataTypeDecimal) vec_to[i] = convertToDecimal(vec_from[i], vec_to.getScale()); else - throw Exception("Unsupported data type in conversion function", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("Unsupported data type in conversion function", ErrorCodes::CANNOT_CONVERT_TYPE); } else vec_to[i] = static_cast(vec_from[i]); From 6a7ed8881ece9478f4793d6a75c335378b37dcb6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 19:39:34 +0300 Subject: [PATCH 108/150] Fix test --- programs/client/Client.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index ff119d89ce6..878eaede2fe 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1583,7 +1583,12 @@ private: if (std::string::npos != embedded_stack_trace_pos && !config().getBool("stacktrace", false)) text.resize(embedded_stack_trace_pos); - std::cerr << "\nReceived exception from server (version " << server_version << "):" << std::endl + /// If we probably have progress bar, we should add additional newline, + /// otherwise exception may display concatenated with the progress bar. + if (need_render_progress) + std::cerr << '\n'; + + std::cerr << "Received exception from server (version " << server_version << "):" << std::endl << "Code: " << e.code() << ". " << text << std::endl; } From c24084a46d85c99caa7b5b6fbc7469bd7fcc4255 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 19:41:45 +0300 Subject: [PATCH 109/150] Fix test --- .../0_stateless/01290_max_execution_speed_distributed.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/01290_max_execution_speed_distributed.reference b/tests/queries/0_stateless/01290_max_execution_speed_distributed.reference index 48355e41fe4..ad0e80d8e69 100644 --- a/tests/queries/0_stateless/01290_max_execution_speed_distributed.reference +++ b/tests/queries/0_stateless/01290_max_execution_speed_distributed.reference @@ -1,4 +1,3 @@ 2000000 1 1 -0 From 387843411d862a4c6b2e3fa53b81e6db6471629b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 19:46:58 +0300 Subject: [PATCH 110/150] Make is more easy --- .../0_stateless/01287_max_execution_speed.reference | 2 -- .../queries/0_stateless/01287_max_execution_speed.sql | 10 +++++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01287_max_execution_speed.reference b/tests/queries/0_stateless/01287_max_execution_speed.reference index 5a3fde429cf..2e85f1f5335 100644 --- a/tests/queries/0_stateless/01287_max_execution_speed.reference +++ b/tests/queries/0_stateless/01287_max_execution_speed.reference @@ -6,5 +6,3 @@ Ok (3) 2000000 1 Ok (4) -1000000 -Ok (5) diff --git a/tests/queries/0_stateless/01287_max_execution_speed.sql b/tests/queries/0_stateless/01287_max_execution_speed.sql index 921a5f77dd5..7e8f6681c84 100644 --- a/tests/queries/0_stateless/01287_max_execution_speed.sql +++ b/tests/queries/0_stateless/01287_max_execution_speed.sql @@ -34,7 +34,11 @@ SET max_execution_speed_bytes = 0; -- Note that 'min_execution_speed' does not count sleeping due to throttling -- with 'max_execution_speed' and similar limits like 'priority' and 'max_network_bandwidth' -SET max_execution_speed = 1000000, min_execution_speed = 2000000; + +-- Note: I have to disable this part of the test because it actually can work slower under sanitizers, +-- with debug builds and in presense of random system hickups in our CI environment. + +--SET max_execution_speed = 1000000, min_execution_speed = 2000000; -- And this query will work despite the fact that the above settings look contradictory. -SELECT count() FROM numbers(1000000); -SELECT 'Ok (5)'; +--SELECT count() FROM numbers(1000000); +--SELECT 'Ok (5)'; From 6604b6aa181f51421e7688986d85453772b79c89 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 20:03:28 +0300 Subject: [PATCH 111/150] Debug bad test "mysql_federated" --- tests/integration/test_mysql_protocol/test.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index f75a168d5db..e2c02239ae9 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -144,7 +144,7 @@ def test_mysql_federated(mysql_server, server_address): node.query('''CREATE TABLE mysql_federated.test (col UInt32) ENGINE = Log''', settings={"password": "123"}) node.query('''INSERT INTO mysql_federated.test VALUES (0), (1), (5)''', settings={"password": "123"}) - code, (_, stderr) = mysql_server.exec_run(''' + code, (stdout, stderr) = mysql_server.exec_run(''' mysql -e "DROP SERVER IF EXISTS clickhouse;" -e "CREATE SERVER clickhouse FOREIGN DATA WRAPPER mysql OPTIONS (USER 'default', PASSWORD '123', HOST '{host}', PORT {port}, DATABASE 'mysql_federated');" @@ -152,6 +152,9 @@ def test_mysql_federated(mysql_server, server_address): -e "CREATE DATABASE mysql_federated;" '''.format(host=server_address, port=server_port), demux=True) + if code != 0: + print(stdout) + print(stderr) assert code == 0 code, (stdout, stderr) = mysql_server.exec_run(''' From ff8a1c04fb7de63b4cc811495900ff834a11138b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 20:53:55 +0300 Subject: [PATCH 112/150] Fix data race in clickhouse-copier --- programs/copier/ClusterCopier.cpp | 42 ++++++++++++++++++------------- programs/copier/ClusterCopier.h | 5 ++-- 2 files changed, 26 insertions(+), 21 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index d50b89738aa..6f9deb30c94 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -4,6 +4,8 @@ #include #include +#include + namespace DB { @@ -177,7 +179,11 @@ void ClusterCopier::discoverTablePartitions(const ConnectionTimeouts & timeouts, ThreadPool thread_pool(num_threads ? num_threads : 2 * getNumberOfPhysicalCPUCores()); for (const TaskShardPtr & task_shard : task_table.all_shards) - thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() { discoverShardPartitions(timeouts, task_shard); }); + thread_pool.scheduleOrThrowOnError([this, timeouts, task_shard]() + { + setThreadName("DiscoverPartns"); + discoverShardPartitions(timeouts, task_shard); + }); LOG_DEBUG(log, "Waiting for {} setup jobs", thread_pool.active()); thread_pool.wait(); @@ -610,7 +616,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t task_table.cluster_push, query_alter_ast_string, nullptr, - &settings_push, + settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -639,7 +645,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t task_table.cluster_push, query_deduplicate_ast_string, nullptr, - &task_cluster->settings_push, + task_cluster->settings_push, PoolMode::GET_MANY); LOG_INFO(log, "Number of shard that executed OPTIMIZE DEDUPLICATE query successfully : {}", toString(num_nodes)); @@ -819,7 +825,7 @@ bool ClusterCopier::tryDropPartitionPiece( size_t num_shards = executeQueryOnCluster( cluster_push, query, nullptr, - &settings_push, + settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -1357,7 +1363,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( LOG_DEBUG(log, "Create destination tables. Query: {}", query); UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, - create_query_push_ast, &task_cluster->settings_push, + create_query_push_ast, task_cluster->settings_push, PoolMode::GET_MANY); LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); } @@ -1480,7 +1486,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( LOG_DEBUG(log, "Create destination tables. Query: {}", query); UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, - create_query_push_ast, &task_cluster->settings_push, + create_query_push_ast, task_cluster->settings_push, PoolMode::GET_MANY); LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); } @@ -1549,7 +1555,7 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table) UInt64 num_nodes = executeQueryOnCluster( cluster_push, query, nullptr, - &settings_push, + settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -1576,7 +1582,7 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT UInt64 num_nodes = executeQueryOnCluster( cluster_push, query, nullptr, - &settings_push, + settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -1789,13 +1795,11 @@ UInt64 ClusterCopier::executeQueryOnCluster( const ClusterPtr & cluster, const String & query, const ASTPtr & query_ast_, - const Settings * settings, + const Settings & current_settings, PoolMode pool_mode, ClusterExecutionMode execution_mode, UInt64 max_successful_executions_per_shard) const { - Settings current_settings = settings ? *settings : task_cluster->settings_common; - auto num_shards = cluster->getShardsInfo().size(); std::vector per_shard_num_successful_replicas(num_shards, 0); @@ -1815,8 +1819,10 @@ UInt64 ClusterCopier::executeQueryOnCluster( std::atomic origin_replicas_number; /// We need to execute query on one replica at least - auto do_for_shard = [&] (UInt64 shard_index) + auto do_for_shard = [&] (UInt64 shard_index, Settings shard_settings) { + setThreadName("QueryForShard"); + const Cluster::ShardInfo & shard = cluster->getShardsInfo().at(shard_index); UInt64 & num_successful_executions = per_shard_num_successful_replicas.at(shard_index); num_successful_executions = 0; @@ -1846,10 +1852,10 @@ UInt64 ClusterCopier::executeQueryOnCluster( /// Will try to make as many as possible queries if (shard.hasRemoteConnections()) { - current_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; + shard_settings.max_parallel_replicas = num_remote_replicas ? num_remote_replicas : 1; - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings).getSaturated(current_settings.max_execution_time); - auto connections = shard.pool->getMany(timeouts, ¤t_settings, pool_mode); + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(shard_settings).getSaturated(shard_settings.max_execution_time); + auto connections = shard.pool->getMany(timeouts, &shard_settings, pool_mode); for (auto & connection : connections) { @@ -1859,7 +1865,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( try { /// CREATE TABLE and DROP PARTITION queries return empty block - RemoteBlockInputStream stream{*connection, query, Block{}, context, ¤t_settings}; + RemoteBlockInputStream stream{*connection, query, Block{}, context, &shard_settings}; NullBlockOutputStream output{Block{}}; copyData(stream, output); @@ -1878,7 +1884,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( ThreadPool thread_pool(std::min(num_shards, getNumberOfPhysicalCPUCores())); for (UInt64 shard_index = 0; shard_index < num_shards; ++shard_index) - thread_pool.scheduleOrThrowOnError([=] { do_for_shard(shard_index); }); + thread_pool.scheduleOrThrowOnError([=, shard_settings = current_settings] { do_for_shard(shard_index, std::move(shard_settings)); }); thread_pool.wait(); } @@ -1898,7 +1904,7 @@ UInt64 ClusterCopier::executeQueryOnCluster( LOG_INFO(log, "There was an error while executing ALTER on each node. Query was executed on {} nodes. But had to be executed on {}", toString(successful_nodes), toString(origin_replicas_number.load())); } - return successful_nodes; } + } diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index 3d6400f51d4..d25cc8c501f 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -15,7 +15,6 @@ namespace DB class ClusterCopier { public: - ClusterCopier(const String & task_path_, const String & host_id_, const String & proxy_database_name_, @@ -187,8 +186,8 @@ protected: UInt64 executeQueryOnCluster( const ClusterPtr & cluster, const String & query, - const ASTPtr & query_ast_ = nullptr, - const Settings * settings = nullptr, + const ASTPtr & query_ast_, + const Settings & current_settings, PoolMode pool_mode = PoolMode::GET_ALL, ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD, UInt64 max_successful_executions_per_shard = 0) const; From a5770fe041548d36b95b45802a543c1bda7e62af Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 30 May 2020 21:05:05 +0300 Subject: [PATCH 113/150] Update PullingPipelineExecutor.cpp --- src/Processors/Executors/PullingPipelineExecutor.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Processors/Executors/PullingPipelineExecutor.cpp b/src/Processors/Executors/PullingPipelineExecutor.cpp index 18722dfdc2f..dc59e0a2f5a 100644 --- a/src/Processors/Executors/PullingPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingPipelineExecutor.cpp @@ -74,10 +74,6 @@ void PullingPipelineExecutor::cancel() /// Cancel execution if it wasn't finished. if (executor) executor->cancel(); - - /// Read all data and finish execution. - /// Chunk chunk; - /// while (pull(chunk)); } Chunk PullingPipelineExecutor::getTotals() From 853872b4043f3a8ef8fa8d4ff67dbb11ef8ea2f8 Mon Sep 17 00:00:00 2001 From: Alexey Ilyukhov Date: Sat, 30 May 2020 21:37:32 +0300 Subject: [PATCH 114/150] Fix style --- src/DataTypes/DataTypeCustomGeo.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index 2f70e9f732a..41a2087d7df 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -32,7 +32,8 @@ public: nestedDataType()->deserializeAsWholeText(column, istr, settings); } - static DataTypePtr nestedDataType() { + static DataTypePtr nestedDataType() + { static auto dataType = DataTypePtr(std::make_unique( DataTypes({std::make_unique(), std::make_unique()}))); return dataType; @@ -55,7 +56,8 @@ public: nestedDataType()->deserializeAsWholeText(column, istr, settings); } - static DataTypePtr nestedDataType() { + static DataTypePtr nestedDataType() + { static auto dataType = DataTypePtr(std::make_unique(DataTypeCustomPointSerialization::nestedDataType())); return dataType; } @@ -77,7 +79,8 @@ public: nestedDataType()->deserializeAsWholeText(column, istr, settings); } - static DataTypePtr nestedDataType() { + static DataTypePtr nestedDataType() + { static auto dataType = DataTypePtr(std::make_unique(DataTypeCustomRingSerialization::nestedDataType())); return dataType; } @@ -99,7 +102,8 @@ public: nestedDataType()->deserializeAsWholeText(column, istr, settings); } - static DataTypePtr nestedDataType() { + static DataTypePtr nestedDataType() + { static auto dataType = DataTypePtr(std::make_unique(DataTypeCustomPolygonSerialization::nestedDataType())); return dataType; } From 43c2a466723660b4ad0f87db194c20f5e7028ae9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 21:57:02 +0300 Subject: [PATCH 115/150] Fix very rare race condition in ThreadPool --- src/Common/ThreadPool.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 7d6935c0383..3e6e31ed3fc 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -234,14 +234,6 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ --scheduled_jobs; } - DB::tryLogCurrentException("ThreadPool", - std::string("Exception in ThreadPool(") + - "max_threads: " + std::to_string(max_threads) - + ", max_free_threads: " + std::to_string(max_free_threads) - + ", queue_size: " + std::to_string(queue_size) - + ", shutdown_on_exception: " + std::to_string(shutdown_on_exception) - + ")."); - job_finished.notify_all(); new_job_or_shutdown.notify_all(); return; From 3574dfe1db3e3d2bdaab1346b948357e6a9a1ec7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 22:07:46 +0300 Subject: [PATCH 116/150] Added test for Distributed table with Seting looking at tables with LowCardinality(String) in presense of memory efficient aggregation setting #6638 --- ...ributed_low_cardinality_memory_efficient.reference | 1 + ...1_distributed_low_cardinality_memory_efficient.sql | 11 +++++++++++ 2 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.reference create mode 100644 tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.sql diff --git a/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.reference b/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.reference new file mode 100644 index 00000000000..257cc5642cb --- /dev/null +++ b/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.reference @@ -0,0 +1 @@ +foo diff --git a/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.sql b/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.sql new file mode 100644 index 00000000000..f0992f2e438 --- /dev/null +++ b/tests/queries/0_stateless/01291_distributed_low_cardinality_memory_efficient.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS data; +DROP TABLE IF EXISTS dist; + +create table data (key String) Engine=Memory(); +create table dist (key LowCardinality(String)) engine=Distributed(test_cluster_two_shards, currentDatabase(), data); +insert into data values ('foo'); +set distributed_aggregation_memory_efficient=1; +select * from dist group by key; + +DROP TABLE data; +DROP TABLE dist; From 09d2f1928781aaa68b858211b259c06bb2190f18 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 22:38:25 +0300 Subject: [PATCH 117/150] Better exception in case when there is shortage of memory mappings --- src/Common/Exception.cpp | 63 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 23e5e0a762e..a81ff2c7328 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -5,8 +5,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -25,6 +27,8 @@ namespace ErrorCodes extern const int STD_EXCEPTION; extern const int UNKNOWN_EXCEPTION; extern const int LOGICAL_ERROR; + extern const int CANNOT_ALLOCATE_MEMORY; + extern const int CANNOT_MREMAP; } @@ -156,6 +160,58 @@ static void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & #endif } + +/** It is possible that the system has enough memory, + * but we have shortage of the number of available memory mappings. + * Provide good diagnostic to user in that case. + */ +static void getNotEnoughMemoryMessage(std::string & msg) +{ +#if defined(__linux__) + try + { + UInt64 max_map_count = 0; + { + ReadBufferFromFile file("/proc/sys/vm/max_map_count"); + readText(max_map_count, file); + } + + UInt64 num_maps = 0; + { + ReadBufferFromFile file("/proc/self/maps"); + while (!file.eof()) + { + char * next_pos = find_first_symbols<'\n'>(file.position(), file.buffer().end()); + file.position() = next_pos; + + if (!file.hasPendingData()) + continue; + + if (*file.position() == '\n') + { + ++num_maps; + ++file.position(); + } + } + } + + if (num_maps > max_map_count * 0.99) + { + msg += fmt::format( + "\nIt looks like that the process is near the limit on number of virtual memory mappings." + "\nCurrent number of mappings (/proc/self/maps): {}." + "\nLimit on number of mappings (/proc/sys/vm/max_map_count): {}." + "\nYou should increase the limit for vm.max_map_count in /etc/sysctl.conf", + num_maps, max_map_count); + } + } + catch (...) + { + msg += "\nCannot obtain additional info about memory usage."; + } +#endif +} + static std::string getExtraExceptionInfo(const std::exception & e) { String msg; @@ -170,6 +226,13 @@ static std::string getExtraExceptionInfo(const std::exception & e) { if (errno_exception->getErrno() == ENOSPC && errno_exception->getPath()) getNoSpaceLeftInfoMessage(errno_exception->getPath().value(), msg); + else if (errno_exception->code() == ErrorCodes::CANNOT_ALLOCATE_MEMORY + || errno_exception->code() == ErrorCodes::CANNOT_MREMAP) + getNotEnoughMemoryMessage(msg); + } + else if (dynamic_cast(&e)) + { + getNotEnoughMemoryMessage(msg); } } catch (...) From 9a41eb14a66fa7c0b7dcab1edee1a6552c3fde7c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 22:40:23 +0300 Subject: [PATCH 118/150] Fix build --- src/Common/Exception.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index a81ff2c7328..3db73631fbb 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -209,6 +209,8 @@ static void getNotEnoughMemoryMessage(std::string & msg) { msg += "\nCannot obtain additional info about memory usage."; } +#else + (void)msg; #endif } From b95b6ec0ee1b92d50abf38c04300da3b17579f04 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 23:02:44 +0300 Subject: [PATCH 119/150] Tested with "trap" function --- src/Common/Exception.cpp | 10 +++++++--- src/Functions/trap.cpp | 21 +++++++++++++++++++++ 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 3db73631fbb..bac30c802c1 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -170,15 +170,18 @@ static void getNotEnoughMemoryMessage(std::string & msg) #if defined(__linux__) try { + static constexpr size_t buf_size = 4096; + char buf[buf_size]; + UInt64 max_map_count = 0; { - ReadBufferFromFile file("/proc/sys/vm/max_map_count"); + ReadBufferFromFile file("/proc/sys/vm/max_map_count", buf_size, -1, buf); readText(max_map_count, file); } UInt64 num_maps = 0; { - ReadBufferFromFile file("/proc/self/maps"); + ReadBufferFromFile file("/proc/self/maps", buf_size, -1, buf); while (!file.eof()) { char * next_pos = find_first_symbols<'\n'>(file.position(), file.buffer().end()); @@ -201,7 +204,8 @@ static void getNotEnoughMemoryMessage(std::string & msg) "\nIt looks like that the process is near the limit on number of virtual memory mappings." "\nCurrent number of mappings (/proc/self/maps): {}." "\nLimit on number of mappings (/proc/sys/vm/max_map_count): {}." - "\nYou should increase the limit for vm.max_map_count in /etc/sysctl.conf", + "\nYou should increase the limit for vm.max_map_count in /etc/sysctl.conf" + "\n", num_maps, max_map_count); } } diff --git a/src/Functions/trap.cpp b/src/Functions/trap.cpp index a7f8d81576d..0f58d73b7fb 100644 --- a/src/Functions/trap.cpp +++ b/src/Functions/trap.cpp @@ -7,11 +7,13 @@ #include #include #include +#include #include #include #include #include +#include namespace DB @@ -132,6 +134,25 @@ public: { (void)context.getCurrentQueryId(); } + else if (mode == "mmap many") + { + std::vector maps; + SCOPE_EXIT( + { + //for (void * map : maps) + // munmap(map, 4096); + }); + + while (true) + { + void * hint = reinterpret_cast( + std::uniform_int_distribution(0x100000000000UL, 0x700000000000UL)(thread_local_rng)); + void * map = mmap(hint, 4096, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); + if (MAP_FAILED == map) + throwFromErrno("Allocator: Cannot mmap", ErrorCodes::CANNOT_ALLOCATE_MEMORY); + maps.push_back(map); + } + } else throw Exception("Unknown trap mode", ErrorCodes::BAD_ARGUMENTS); } From 1b0c9fae6cfaa8e3bfdfc6ba494179f2f4fc14ee Mon Sep 17 00:00:00 2001 From: Alexey Ilyukhov Date: Sat, 30 May 2020 23:15:35 +0300 Subject: [PATCH 120/150] Fix style --- src/DataTypes/DataTypeCustomGeo.cpp | 31 ++++++++--------------------- 1 file changed, 8 insertions(+), 23 deletions(-) diff --git a/src/DataTypes/DataTypeCustomGeo.cpp b/src/DataTypes/DataTypeCustomGeo.cpp index 41a2087d7df..baffde700c8 100644 --- a/src/DataTypes/DataTypeCustomGeo.cpp +++ b/src/DataTypes/DataTypeCustomGeo.cpp @@ -15,13 +15,7 @@ namespace class DataTypeCustomPointSerialization : public DataTypeCustomSimpleTextSerialization { -private: - DataTypePtr tuple; - public: - DataTypeCustomPointSerialization() - {} - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { nestedDataType()->serializeAsText(column, row_num, ostr, settings); @@ -34,18 +28,15 @@ public: static DataTypePtr nestedDataType() { - static auto dataType = DataTypePtr(std::make_unique( + static auto data_type = DataTypePtr(std::make_unique( DataTypes({std::make_unique(), std::make_unique()}))); - return dataType; + return data_type; } }; class DataTypeCustomRingSerialization : public DataTypeCustomSimpleTextSerialization { public: - DataTypeCustomRingSerialization() - {} - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { nestedDataType()->serializeAsText(column, row_num, ostr, settings); @@ -58,17 +49,14 @@ public: static DataTypePtr nestedDataType() { - static auto dataType = DataTypePtr(std::make_unique(DataTypeCustomPointSerialization::nestedDataType())); - return dataType; + static auto data_type = DataTypePtr(std::make_unique(DataTypeCustomPointSerialization::nestedDataType())); + return data_type; } }; class DataTypeCustomPolygonSerialization : public DataTypeCustomSimpleTextSerialization { public: - DataTypeCustomPolygonSerialization() - {} - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { nestedDataType()->serializeAsText(column, row_num, ostr, settings); @@ -81,17 +69,14 @@ public: static DataTypePtr nestedDataType() { - static auto dataType = DataTypePtr(std::make_unique(DataTypeCustomRingSerialization::nestedDataType())); - return dataType; + static auto data_type = DataTypePtr(std::make_unique(DataTypeCustomRingSerialization::nestedDataType())); + return data_type; } }; class DataTypeCustomMultiPolygonSerialization : public DataTypeCustomSimpleTextSerialization { public: - DataTypeCustomMultiPolygonSerialization() - {} - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override { nestedDataType()->serializeAsText(column, row_num, ostr, settings); @@ -104,8 +89,8 @@ public: static DataTypePtr nestedDataType() { - static auto dataType = DataTypePtr(std::make_unique(DataTypeCustomPolygonSerialization::nestedDataType())); - return dataType; + static auto data_type = DataTypePtr(std::make_unique(DataTypeCustomPolygonSerialization::nestedDataType())); + return data_type; } }; From 26100b64deb4718be61965ee4f517fb0bed7edb9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 30 May 2020 23:30:08 +0300 Subject: [PATCH 121/150] Remove function parameter --- programs/copier/ClusterCopier.cpp | 24 ++++-------------------- programs/copier/ClusterCopier.h | 1 - 2 files changed, 4 insertions(+), 21 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index 6f9deb30c94..bc32f586a01 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -615,7 +615,6 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t size_t num_nodes = executeQueryOnCluster( task_table.cluster_push, query_alter_ast_string, - nullptr, settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -644,7 +643,6 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t UInt64 num_nodes = executeQueryOnCluster( task_table.cluster_push, query_deduplicate_ast_string, - nullptr, task_cluster->settings_push, PoolMode::GET_MANY); @@ -824,7 +822,6 @@ bool ClusterCopier::tryDropPartitionPiece( /// We have to drop partition_piece on each replica size_t num_shards = executeQueryOnCluster( cluster_push, query, - nullptr, settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -1362,9 +1359,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( String query = queryToString(create_query_push_ast); LOG_DEBUG(log, "Create destination tables. Query: {}", query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, - create_query_push_ast, task_cluster->settings_push, - PoolMode::GET_MANY); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY); LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); } @@ -1485,9 +1480,7 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( String query = queryToString(create_query_push_ast); LOG_DEBUG(log, "Create destination tables. Query: {}", query); - UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, - create_query_push_ast, task_cluster->settings_push, - PoolMode::GET_MANY); + UInt64 shards = executeQueryOnCluster(task_table.cluster_push, query, task_cluster->settings_push, PoolMode::GET_MANY); LOG_DEBUG(log, "Destination tables {} have been created on {} shards of {}", getQuotedTable(task_table.table_push), shards, task_table.cluster_push->getShardCount()); } catch (...) @@ -1554,7 +1547,6 @@ void ClusterCopier::dropHelpingTables(const TaskTable & task_table) /// We have to drop partition_piece on each replica UInt64 num_nodes = executeQueryOnCluster( cluster_push, query, - nullptr, settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -1581,7 +1573,6 @@ void ClusterCopier::dropParticularPartitionPieceFromAllHelpingTables(const TaskT /// We have to drop partition_piece on each replica UInt64 num_nodes = executeQueryOnCluster( cluster_push, query, - nullptr, settings_push, PoolMode::GET_MANY, ClusterExecutionMode::ON_EACH_NODE); @@ -1794,7 +1785,6 @@ bool ClusterCopier::checkPresentPartitionPiecesOnCurrentShard(const ConnectionTi UInt64 ClusterCopier::executeQueryOnCluster( const ClusterPtr & cluster, const String & query, - const ASTPtr & query_ast_, const Settings & current_settings, PoolMode pool_mode, ClusterExecutionMode execution_mode, @@ -1803,14 +1793,8 @@ UInt64 ClusterCopier::executeQueryOnCluster( auto num_shards = cluster->getShardsInfo().size(); std::vector per_shard_num_successful_replicas(num_shards, 0); - ASTPtr query_ast; - if (query_ast_ == nullptr) - { - ParserQuery p_query(query.data() + query.size()); - query_ast = parseQuery(p_query, query, current_settings.max_query_size, current_settings.max_parser_depth); - } - else - query_ast = query_ast_; + ParserQuery p_query(query.data() + query.size()); + ASTPtr query_ast = parseQuery(p_query, query, current_settings.max_query_size, current_settings.max_parser_depth); /// We will have to execute query on each replica of a shard. if (execution_mode == ClusterExecutionMode::ON_EACH_NODE) diff --git a/programs/copier/ClusterCopier.h b/programs/copier/ClusterCopier.h index d25cc8c501f..beaf247dfc8 100644 --- a/programs/copier/ClusterCopier.h +++ b/programs/copier/ClusterCopier.h @@ -186,7 +186,6 @@ protected: UInt64 executeQueryOnCluster( const ClusterPtr & cluster, const String & query, - const ASTPtr & query_ast_, const Settings & current_settings, PoolMode pool_mode = PoolMode::GET_ALL, ClusterExecutionMode execution_mode = ClusterExecutionMode::ON_EACH_SHARD, From 383962e47ce9a4702782514f45c02bb45b8283a2 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 30 May 2020 23:45:30 +0300 Subject: [PATCH 122/150] Update trap.cpp --- src/Functions/trap.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/trap.cpp b/src/Functions/trap.cpp index 0f58d73b7fb..94aa56919b4 100644 --- a/src/Functions/trap.cpp +++ b/src/Functions/trap.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; + extern const int CANNOT_ALLOCATE_MEMORY; } From 5aff138956784684a57b312e7abf15eed8c954c4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 00:35:52 +0300 Subject: [PATCH 123/150] Preparation for structured logging --- src/Client/Connection.cpp | 24 ++++++------- src/Common/AlignedBuffer.cpp | 4 +-- src/Common/Allocator.h | 18 +++++----- src/Common/ArrayCache.h | 4 +-- src/Common/AutoArray.h | 2 +- src/Common/Exception.cpp | 15 ++++---- src/Common/MemoryTracker.cpp | 4 +-- src/Common/PipeFDs.cpp | 2 +- src/Common/formatReadable.h | 34 +++++++++++++++++++ .../AggregatingBlockInputStream.cpp | 2 +- src/DataStreams/ColumnGathererStream.cpp | 2 +- .../MergeSortingBlockInputStream.cpp | 2 +- .../MergingSortedBlockInputStream.cpp | 2 +- .../ParallelAggregatingBlockInputStream.cpp | 10 +++--- src/DataStreams/SizeLimits.cpp | 4 +-- src/Disks/DiskLocal.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 2 +- src/Disks/VolumeJBOD.cpp | 4 +-- src/IO/MMapReadBufferFromFileDescriptor.cpp | 4 +-- src/IO/tests/read_float_perf.cpp | 3 +- src/Interpreters/Aggregator.cpp | 22 ++++++------ src/Interpreters/executeQuery.cpp | 4 +-- .../Formats/Impl/MySQLOutputFormat.cpp | 15 ++++---- .../Merges/MergingSortedTransform.cpp | 2 +- .../Transforms/AggregatingTransform.cpp | 6 ++-- .../Transforms/MergeSortingTransform.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 3 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 17 +++++++--- .../MergeTree/MergeTreePartsMover.cpp | 2 +- src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 17 +++++++--- src/Storages/StorageSet.cpp | 2 +- 32 files changed, 146 insertions(+), 92 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 6c1f7f8e953..367d4bab1dc 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -508,18 +508,18 @@ void Connection::sendScalarsData(Scalars & data) "Sent data for {} scalars, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), compressed {} times to {} ({}/sec.)", data.size(), rows, elapsed, static_cast(rows / watch.elapsedSeconds()), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()), + ReadableSize(maybe_compressed_out_bytes), + ReadableSize(maybe_compressed_out_bytes / watch.elapsedSeconds()), static_cast(maybe_compressed_out_bytes) / out_bytes, - formatReadableSizeWithBinarySuffix(out_bytes), - formatReadableSizeWithBinarySuffix(out_bytes / watch.elapsedSeconds())); + ReadableSize(out_bytes), + ReadableSize(out_bytes / watch.elapsedSeconds())); else LOG_DEBUG(log_wrapper.get(), "Sent data for {} scalars, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), no compression.", data.size(), rows, elapsed, static_cast(rows / watch.elapsedSeconds()), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds())); + ReadableSize(maybe_compressed_out_bytes), + ReadableSize(maybe_compressed_out_bytes / watch.elapsedSeconds())); } namespace @@ -612,18 +612,18 @@ void Connection::sendExternalTablesData(ExternalTablesData & data) "Sent data for {} external tables, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), compressed {} times to {} ({}/sec.)", data.size(), rows, elapsed, static_cast(rows / watch.elapsedSeconds()), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds()), + ReadableSize(maybe_compressed_out_bytes), + ReadableSize(maybe_compressed_out_bytes / watch.elapsedSeconds()), static_cast(maybe_compressed_out_bytes) / out_bytes, - formatReadableSizeWithBinarySuffix(out_bytes), - formatReadableSizeWithBinarySuffix(out_bytes / watch.elapsedSeconds())); + ReadableSize(out_bytes), + ReadableSize(out_bytes / watch.elapsedSeconds())); else LOG_DEBUG(log_wrapper.get(), "Sent data for {} external tables, total {} rows in {} sec., {} rows/sec., {} ({}/sec.), no compression.", data.size(), rows, elapsed, static_cast(rows / watch.elapsedSeconds()), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes), - formatReadableSizeWithBinarySuffix(maybe_compressed_out_bytes / watch.elapsedSeconds())); + ReadableSize(maybe_compressed_out_bytes), + ReadableSize(maybe_compressed_out_bytes / watch.elapsedSeconds())); } std::optional Connection::getResolvedAddress() const diff --git a/src/Common/AlignedBuffer.cpp b/src/Common/AlignedBuffer.cpp index 6d4775ebadf..f1d3f98ff3a 100644 --- a/src/Common/AlignedBuffer.cpp +++ b/src/Common/AlignedBuffer.cpp @@ -18,8 +18,8 @@ void AlignedBuffer::alloc(size_t size, size_t alignment) void * new_buf; int res = ::posix_memalign(&new_buf, std::max(alignment, sizeof(void*)), size); if (0 != res) - throwFromErrno("Cannot allocate memory (posix_memalign), size: " - + formatReadableSizeWithBinarySuffix(size) + ", alignment: " + formatReadableSizeWithBinarySuffix(alignment) + ".", + throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign), size: {}, alignment: {}.", + ReadableSize(size), ReadableSize(alignment)), ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); buf = new_buf; } diff --git a/src/Common/Allocator.h b/src/Common/Allocator.h index 43d7e67c4bb..ead456f935e 100644 --- a/src/Common/Allocator.h +++ b/src/Common/Allocator.h @@ -129,7 +129,7 @@ public: void * new_buf = ::realloc(buf, new_size); if (nullptr == new_buf) - DB::throwFromErrno("Allocator: Cannot realloc from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + DB::throwFromErrno(fmt::format("Allocator: Cannot realloc from {} to {}.", ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); buf = new_buf; if constexpr (clear_memory) @@ -145,7 +145,8 @@ public: buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE, PROT_READ | PROT_WRITE, mmap_flags, -1, 0); if (MAP_FAILED == buf) - DB::throwFromErrno("Allocator: Cannot mremap memory chunk from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_MREMAP); + DB::throwFromErrno(fmt::format("Allocator: Cannot mremap memory chunk from {} to {}.", + ReadableSize(old_size), ReadableSize(new_size)), DB::ErrorCodes::CANNOT_MREMAP); /// No need for zero-fill, because mmap guarantees it. } @@ -201,13 +202,13 @@ private: if (size >= MMAP_THRESHOLD) { if (alignment > MMAP_MIN_ALIGNMENT) - throw DB::Exception("Too large alignment " + formatReadableSizeWithBinarySuffix(alignment) + ": more than page size when allocating " - + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::BAD_ARGUMENTS); + throw DB::Exception(fmt::format("Too large alignment {}: more than page size when allocating {}.", + ReadableSize(alignment), ReadableSize(size)), DB::ErrorCodes::BAD_ARGUMENTS); buf = mmap(getMmapHint(), size, PROT_READ | PROT_WRITE, mmap_flags, -1, 0); if (MAP_FAILED == buf) - DB::throwFromErrno("Allocator: Cannot mmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + DB::throwFromErrno(fmt::format("Allocator: Cannot mmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); /// No need for zero-fill, because mmap guarantees it. } @@ -221,7 +222,7 @@ private: buf = ::malloc(size); if (nullptr == buf) - DB::throwFromErrno("Allocator: Cannot malloc " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + DB::throwFromErrno(fmt::format("Allocator: Cannot malloc {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); } else { @@ -229,7 +230,8 @@ private: int res = posix_memalign(&buf, alignment, size); if (0 != res) - DB::throwFromErrno("Cannot allocate memory (posix_memalign) " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); + DB::throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(size)), + DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); if constexpr (clear_memory) memset(buf, 0, size); @@ -243,7 +245,7 @@ private: if (size >= MMAP_THRESHOLD) { if (0 != munmap(buf, size)) - DB::throwFromErrno("Allocator: Cannot munmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_MUNMAP); + DB::throwFromErrno(fmt::format("Allocator: Cannot munmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_MUNMAP); } else { diff --git a/src/Common/ArrayCache.h b/src/Common/ArrayCache.h index 304ab8e03c3..5e3af09a2fd 100644 --- a/src/Common/ArrayCache.h +++ b/src/Common/ArrayCache.h @@ -177,13 +177,13 @@ private: { ptr = mmap(address_hint, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); if (MAP_FAILED == ptr) - DB::throwFromErrno("Allocator: Cannot mmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + DB::throwFromErrno(fmt::format("Allocator: Cannot mmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); } ~Chunk() { if (ptr && 0 != munmap(ptr, size)) - DB::throwFromErrno("Allocator: Cannot munmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_MUNMAP); + DB::throwFromErrno(fmt::format("Allocator: Cannot munmap {}.", ReadableSize(size)), DB::ErrorCodes::CANNOT_MUNMAP); } Chunk(Chunk && other) : ptr(other.ptr), size(other.size) diff --git a/src/Common/AutoArray.h b/src/Common/AutoArray.h index 92e8d45b07b..a39706baa3d 100644 --- a/src/Common/AutoArray.h +++ b/src/Common/AutoArray.h @@ -278,7 +278,7 @@ private: void * new_data = nullptr; int res = posix_memalign(&new_data, alignment, prefix_size + new_size * sizeof(T)); if (0 != res) - throwFromErrno("Cannot allocate memory (posix_memalign) " + formatReadableSizeWithBinarySuffix(new_size) + ".", + throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(new_size)), ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); data_ptr = static_cast(new_data); diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 23e5e0a762e..0233fb00a43 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -144,13 +144,16 @@ static void getNoSpaceLeftInfoMessage(std::filesystem::path path, std::string & path = path.parent_path(); auto fs = getStatVFS(path); - msg += "\nTotal space: " + formatReadableSizeWithBinarySuffix(fs.f_blocks * fs.f_bsize) - + "\nAvailable space: " + formatReadableSizeWithBinarySuffix(fs.f_bavail * fs.f_bsize) - + "\nTotal inodes: " + formatReadableQuantity(fs.f_files) - + "\nAvailable inodes: " + formatReadableQuantity(fs.f_favail); - auto mount_point = getMountPoint(path).string(); - msg += "\nMount point: " + mount_point; + + fmt::format_to(std::back_inserter(msg), + "\nTotal space: {}\nAvailable space: {}\nTotal inodes: {}\nAvailable inodes: {}\nMount point: {}", + ReadableSize(fs.f_blocks * fs.f_bsize), + ReadableSize(fs.f_bavail * fs.f_bsize), + formatReadableQuantity(fs.f_files), + formatReadableQuantity(fs.f_favail), + mount_point); + #if defined(__linux__) msg += "\nFilesystem: " + getFilesystemName(mount_point); #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 16d166c191b..54e991da41f 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -50,13 +50,13 @@ MemoryTracker::~MemoryTracker() void MemoryTracker::logPeakMemoryUsage() const { const auto * description = description_ptr.load(std::memory_order_relaxed); - LOG_DEBUG(&Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), formatReadableSizeWithBinarySuffix(peak)); + LOG_DEBUG(&Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak)); } void MemoryTracker::logMemoryUsage(Int64 current) const { const auto * description = description_ptr.load(std::memory_order_relaxed); - LOG_DEBUG(&Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), formatReadableSizeWithBinarySuffix(current)); + LOG_DEBUG(&Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current)); } diff --git a/src/Common/PipeFDs.cpp b/src/Common/PipeFDs.cpp index cec8cb1cf29..1f57234534f 100644 --- a/src/Common/PipeFDs.cpp +++ b/src/Common/PipeFDs.cpp @@ -102,7 +102,7 @@ void LazyPipeFDs::tryIncreaseSize(int desired_size) if (-1 == fcntl(fds_rw[1], F_SETPIPE_SZ, pipe_size * 2) && errno != EPERM) throwFromErrno("Cannot increase pipe capacity to " + std::to_string(pipe_size * 2), ErrorCodes::CANNOT_FCNTL); - LOG_TRACE(log, "Pipe capacity is {}", formatReadableSizeWithBinarySuffix(std::min(pipe_size, desired_size))); + LOG_TRACE(log, "Pipe capacity is {}", ReadableSize(std::min(pipe_size, desired_size))); } #else (void)desired_size; diff --git a/src/Common/formatReadable.h b/src/Common/formatReadable.h index 87d24b75c20..1c46bd8af09 100644 --- a/src/Common/formatReadable.h +++ b/src/Common/formatReadable.h @@ -1,6 +1,8 @@ #pragma once #include +#include + namespace DB { @@ -20,3 +22,35 @@ std::string formatReadableSizeWithDecimalSuffix(double value, int precision = 2) /// Prints the number as 123.45 billion. void formatReadableQuantity(double value, DB::WriteBuffer & out, int precision = 2); std::string formatReadableQuantity(double value, int precision = 2); + + +/// Wrapper around value. If used with fmt library (e.g. for log messages), +/// value is automatically formatted as size with binary suffix. +struct ReadableSize +{ + double value; + explicit ReadableSize(double value_) : value(value_) {} +}; + +/// See https://fmt.dev/latest/api.html#formatting-user-defined-types +template <> +struct fmt::formatter +{ + constexpr auto parse(format_parse_context & ctx) + { + auto it = ctx.begin(); + auto end = ctx.end(); + + /// Only support {}. + if (it != end && *it != '}') + throw format_error("invalid format"); + + return it; + } + + template + auto format(const ReadableSize & size, FormatContext & ctx) + { + return format_to(ctx.out(), "{}", formatReadableSizeWithBinarySuffix(size.value)); + } +}; diff --git a/src/DataStreams/AggregatingBlockInputStream.cpp b/src/DataStreams/AggregatingBlockInputStream.cpp index 7c891271468..150f794ca59 100644 --- a/src/DataStreams/AggregatingBlockInputStream.cpp +++ b/src/DataStreams/AggregatingBlockInputStream.cpp @@ -60,7 +60,7 @@ Block AggregatingBlockInputStream::readImpl() input_streams.emplace_back(temporary_inputs.back()->block_in); } - LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), formatReadableSizeWithBinarySuffix(files.sum_size_compressed), formatReadableSizeWithBinarySuffix(files.sum_size_uncompressed)); + LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), ReadableSize(files.sum_size_compressed), ReadableSize(files.sum_size_uncompressed)); impl = std::make_unique(input_streams, params, final, 1, 1); } diff --git a/src/DataStreams/ColumnGathererStream.cpp b/src/DataStreams/ColumnGathererStream.cpp index b98ee96d26a..4922a10b8a3 100644 --- a/src/DataStreams/ColumnGathererStream.cpp +++ b/src/DataStreams/ColumnGathererStream.cpp @@ -105,7 +105,7 @@ void ColumnGathererStream::readSuffixImpl() else LOG_DEBUG(log, "Gathered column {} ({} bytes/elem.) in {} sec., {} rows/sec., {}/sec.", column_name, static_cast(profile_info.bytes) / profile_info.rows, seconds, - profile_info.rows / seconds, formatReadableSizeWithBinarySuffix(profile_info.bytes / seconds)); + profile_info.rows / seconds, ReadableSize(profile_info.bytes / seconds)); } } diff --git a/src/DataStreams/MergeSortingBlockInputStream.cpp b/src/DataStreams/MergeSortingBlockInputStream.cpp index 587f21d2467..5e1fbe599b7 100644 --- a/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -264,7 +264,7 @@ void MergeSortingBlockInputStream::remerge() } merger.readSuffix(); - LOG_DEBUG(log, "Memory usage is lowered from {} to {}", formatReadableSizeWithBinarySuffix(sum_bytes_in_blocks), formatReadableSizeWithBinarySuffix(new_sum_bytes_in_blocks)); + LOG_DEBUG(log, "Memory usage is lowered from {} to {}", ReadableSize(sum_bytes_in_blocks), ReadableSize(new_sum_bytes_in_blocks)); /// If the memory consumption was not lowered enough - we will not perform remerge anymore. 2 is a guess. if (new_sum_bytes_in_blocks * 2 > sum_bytes_in_blocks) diff --git a/src/DataStreams/MergingSortedBlockInputStream.cpp b/src/DataStreams/MergingSortedBlockInputStream.cpp index 877213294ec..703ceff4c8f 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -269,7 +269,7 @@ void MergingSortedBlockInputStream::readSuffixImpl() LOG_DEBUG(log, "Merge sorted {} blocks, {} rows in {} sec., {} rows/sec., {}/sec", profile_info.blocks, profile_info.rows, seconds, profile_info.rows / seconds, - formatReadableSizeWithBinarySuffix(profile_info.bytes / seconds)); + ReadableSize(profile_info.bytes / seconds)); } } diff --git a/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index aeda8ea5be9..611059c1443 100644 --- a/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -82,7 +82,7 @@ Block ParallelAggregatingBlockInputStream::readImpl() input_streams.emplace_back(temporary_inputs.back()->block_in); } - LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), formatReadableSizeWithBinarySuffix(files.sum_size_compressed), formatReadableSizeWithBinarySuffix(files.sum_size_uncompressed)); + LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), ReadableSize(files.sum_size_compressed), ReadableSize(files.sum_size_uncompressed)); impl = std::make_unique( input_streams, params, final, temporary_data_merge_threads, temporary_data_merge_threads); @@ -178,16 +178,16 @@ void ParallelAggregatingBlockInputStream::execute() { size_t rows = many_data[i]->size(); LOG_TRACE(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)", - threads_data[i].src_rows, rows, formatReadableSizeWithBinarySuffix(threads_data[i].src_bytes), + threads_data[i].src_rows, rows, ReadableSize(threads_data[i].src_bytes), elapsed_seconds, threads_data[i].src_rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(threads_data[i].src_bytes / elapsed_seconds)); + ReadableSize(threads_data[i].src_bytes / elapsed_seconds)); total_src_rows += threads_data[i].src_rows; total_src_bytes += threads_data[i].src_bytes; } LOG_TRACE(log, "Total aggregated. {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)", - total_src_rows, formatReadableSizeWithBinarySuffix(total_src_bytes), elapsed_seconds, - total_src_rows / elapsed_seconds, formatReadableSizeWithBinarySuffix(total_src_bytes / elapsed_seconds)); + total_src_rows, ReadableSize(total_src_bytes), elapsed_seconds, + total_src_rows / elapsed_seconds, ReadableSize(total_src_bytes / elapsed_seconds)); /// If there was no data, and we aggregate without keys, we must return single row with the result of empty aggregation. /// To do this, we pass a block with zero rows to aggregate. diff --git a/src/DataStreams/SizeLimits.cpp b/src/DataStreams/SizeLimits.cpp index ed57f66d21a..06dde923e55 100644 --- a/src/DataStreams/SizeLimits.cpp +++ b/src/DataStreams/SizeLimits.cpp @@ -16,8 +16,8 @@ bool SizeLimits::check(UInt64 rows, UInt64 bytes, const char * what, int too_man + ", current rows: " + formatReadableQuantity(rows), too_many_rows_exception_code); if (max_bytes && bytes > max_bytes) - throw Exception("Limit for " + std::string(what) + " exceeded, max bytes: " + formatReadableSizeWithBinarySuffix(max_bytes) - + ", current bytes: " + formatReadableSizeWithBinarySuffix(bytes), too_many_bytes_exception_code); + throw Exception(fmt::format("Limit for {} exceeded, max bytes: {}, current bytes: {}", + std::string(what), ReadableSize(max_bytes), ReadableSize(bytes)), too_many_bytes_exception_code); return true; } diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index cea900d5bad..d7f1df5ba3b 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -101,7 +101,7 @@ bool DiskLocal::tryReserve(UInt64 bytes) if (unreserved_space >= bytes) { LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving {} on disk {}, having unreserved {}.", - formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space)); + ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space)); ++reservation_count; reserved_bytes += bytes; return true; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 4b36deb7e98..2d48db8bc50 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -617,7 +617,7 @@ bool DiskS3::tryReserve(UInt64 bytes) if (unreserved_space >= bytes) { LOG_DEBUG(&Logger::get("DiskS3"), "Reserving {} on disk {}, having unreserved {}.", - formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space)); + ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space)); ++reservation_count; reserved_bytes += bytes; return true; diff --git a/src/Disks/VolumeJBOD.cpp b/src/Disks/VolumeJBOD.cpp index b9eb8038cac..eeea763229d 100644 --- a/src/Disks/VolumeJBOD.cpp +++ b/src/Disks/VolumeJBOD.cpp @@ -48,11 +48,11 @@ VolumeJBOD::VolumeJBOD( max_data_part_size = static_cast(sum_size * ratio / disks.size()); for (size_t i = 0; i < disks.size(); ++i) if (sizes[i] < max_data_part_size) - LOG_WARNING(logger, "Disk {} on volume {} have not enough space ({}) for containing part the size of max_data_part_size ({})", backQuote(disks[i]->getName()), backQuote(config_prefix), formatReadableSizeWithBinarySuffix(sizes[i]), formatReadableSizeWithBinarySuffix(max_data_part_size)); + LOG_WARNING(logger, "Disk {} on volume {} have not enough space ({}) for containing part the size of max_data_part_size ({})", backQuote(disks[i]->getName()), backQuote(config_prefix), ReadableSize(sizes[i]), ReadableSize(max_data_part_size)); } static constexpr UInt64 MIN_PART_SIZE = 8u * 1024u * 1024u; if (max_data_part_size != 0 && max_data_part_size < MIN_PART_SIZE) - LOG_WARNING(logger, "Volume {} max_data_part_size is too low ({} < {})", backQuote(name), formatReadableSizeWithBinarySuffix(max_data_part_size), formatReadableSizeWithBinarySuffix(MIN_PART_SIZE)); + LOG_WARNING(logger, "Volume {} max_data_part_size is too low ({} < {})", backQuote(name), ReadableSize(max_data_part_size), ReadableSize(MIN_PART_SIZE)); } DiskPtr VolumeJBOD::getNextDisk() diff --git a/src/IO/MMapReadBufferFromFileDescriptor.cpp b/src/IO/MMapReadBufferFromFileDescriptor.cpp index 027b95bc022..bbeec8959b4 100644 --- a/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -34,7 +34,7 @@ void MMapReadBufferFromFileDescriptor::init(int fd_, size_t offset, size_t lengt { void * buf = mmap(nullptr, length, PROT_READ, MAP_PRIVATE, fd, offset); if (MAP_FAILED == buf) - throwFromErrno("MMapReadBufferFromFileDescriptor: Cannot mmap " + formatReadableSizeWithBinarySuffix(length) + ".", + throwFromErrno(fmt::format("MMapReadBufferFromFileDescriptor: Cannot mmap {}.", ReadableSize(length)), ErrorCodes::CANNOT_ALLOCATE_MEMORY); BufferBase::set(static_cast(buf), length, 0); @@ -84,7 +84,7 @@ MMapReadBufferFromFileDescriptor::~MMapReadBufferFromFileDescriptor() void MMapReadBufferFromFileDescriptor::finish() { if (0 != munmap(internalBuffer().begin(), length)) - throwFromErrno("MMapReadBufferFromFileDescriptor: Cannot munmap " + formatReadableSizeWithBinarySuffix(length) + ".", + throwFromErrno(fmt::format("MMapReadBufferFromFileDescriptor: Cannot munmap {}.", ReadableSize(length)), ErrorCodes::CANNOT_MUNMAP); length = 0; diff --git a/src/IO/tests/read_float_perf.cpp b/src/IO/tests/read_float_perf.cpp index 8a870d4a960..aa0fa8ad15d 100644 --- a/src/IO/tests/read_float_perf.cpp +++ b/src/IO/tests/read_float_perf.cpp @@ -56,7 +56,8 @@ void NO_INLINE loop(ReadBuffer & in, WriteBuffer & out) } watch.stop(); - out << "Read in " << watch.elapsedSeconds() << " sec, " << formatReadableSizeWithBinarySuffix(in.count() / watch.elapsedSeconds()) << "/sec, result = " << sum << "\n"; + out << "Read in " << watch.elapsedSeconds() << " sec, " + << formatReadableSizeWithBinarySuffix(in.count() / watch.elapsedSeconds()) << "/sec, result = " << sum << "\n"; } diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 98cb1018178..7de1445c960 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -768,14 +768,14 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, co " ({} rows/sec., {}/sec. uncompressed, {}/sec. compressed)", elapsed_seconds, rows, - formatReadableSizeWithBinarySuffix(uncompressed_bytes), - formatReadableSizeWithBinarySuffix(compressed_bytes), + ReadableSize(uncompressed_bytes), + ReadableSize(compressed_bytes), uncompressed_bytes / rows, compressed_bytes / rows, uncompressed_bytes / compressed_bytes, rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(uncompressed_bytes / elapsed_seconds), - formatReadableSizeWithBinarySuffix(compressed_bytes / elapsed_seconds)); + ReadableSize(uncompressed_bytes / elapsed_seconds), + ReadableSize(compressed_bytes / elapsed_seconds)); } void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants) { @@ -871,7 +871,7 @@ void Aggregator::writeToTemporaryFileImpl( /// `data_variants` will not destroy them in the destructor, they are now owned by ColumnAggregateFunction objects. data_variants.aggregator = nullptr; - LOG_TRACE(log, "Max size of temporary block: {} rows, {}.", max_temporary_block_size_rows, formatReadableSizeWithBinarySuffix(max_temporary_block_size_bytes)); + LOG_TRACE(log, "Max size of temporary block: {} rows, {}.", max_temporary_block_size_rows, ReadableSize(max_temporary_block_size_bytes)); } @@ -943,9 +943,9 @@ void Aggregator::execute(const BlockInputStreamPtr & stream, AggregatedDataVaria size_t rows = result.sizeWithoutOverflowRow(); LOG_TRACE(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)", - src_rows, rows, formatReadableSizeWithBinarySuffix(src_bytes), + src_rows, rows, ReadableSize(src_bytes), elapsed_seconds, src_rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(src_bytes / elapsed_seconds)); + ReadableSize(src_bytes / elapsed_seconds)); } @@ -1313,9 +1313,9 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b double elapsed_seconds = watch.elapsedSeconds(); LOG_TRACE(log, "Converted aggregated data to blocks. {} rows, {} in {} sec. ({} rows/sec., {}/sec.)", - rows, formatReadableSizeWithBinarySuffix(bytes), + rows, ReadableSize(bytes), elapsed_seconds, rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(bytes / elapsed_seconds)); + ReadableSize(bytes / elapsed_seconds)); return blocks; } @@ -2178,9 +2178,9 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final) size_t bytes = block.bytes(); double elapsed_seconds = watch.elapsedSeconds(); LOG_TRACE(log, "Merged partially aggregated blocks. {} rows, {}. in {} sec. ({} rows/sec., {}/sec.)", - rows, formatReadableSizeWithBinarySuffix(bytes), + rows, ReadableSize(bytes), elapsed_seconds, rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(bytes / elapsed_seconds)); + ReadableSize(bytes / elapsed_seconds)); if (isCancelled()) return {}; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 8fc799d0b48..e825c1949cf 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -506,9 +506,9 @@ static std::tuple executeQueryImpl( if (elem.read_rows != 0) { LOG_INFO(&Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", - elem.read_rows, formatReadableSizeWithBinarySuffix(elem.read_bytes), elapsed_seconds, + elem.read_rows, ReadableSize(elem.read_bytes), elapsed_seconds, static_cast(elem.read_rows / elapsed_seconds), - formatReadableSizeWithBinarySuffix(elem.read_bytes / elapsed_seconds)); + ReadableSize(elem.read_bytes / elapsed_seconds)); } elem.thread_ids = std::move(info.thread_ids); diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 8f1967d3704..94dfc3dc2c7 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -60,24 +60,25 @@ void MySQLOutputFormat::consume(Chunk chunk) void MySQLOutputFormat::finalize() { size_t affected_rows = 0; - std::stringstream human_readable_info; + std::string human_readable_info; if (QueryStatus * process_list_elem = context->getProcessListElement()) { CurrentThread::finalizePerformanceCounters(); QueryStatusInfo info = process_list_elem->getInfo(); affected_rows = info.written_rows; - human_readable_info << std::fixed << std::setprecision(3) - << "Read " << info.read_rows << " rows, " << formatReadableSizeWithBinarySuffix(info.read_bytes) << " in " << info.elapsed_seconds << " sec., " - << static_cast(info.read_rows / info.elapsed_seconds) << " rows/sec., " - << formatReadableSizeWithBinarySuffix(info.read_bytes / info.elapsed_seconds) << "/sec."; + human_readable_info = fmt::format( + "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", + info.read_rows, ReadableSize(info.read_bytes), info.elapsed_seconds, + static_cast(info.read_rows / info.elapsed_seconds), + ReadableSize(info.read_bytes / info.elapsed_seconds)); } const auto & header = getPort(PortKind::Main).getHeader(); if (header.columns() == 0) - packet_sender->sendPacket(OK_Packet(0x0, context->mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true); + packet_sender->sendPacket(OK_Packet(0x0, context->mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info), true); else if (context->mysql.client_capabilities & CLIENT_DEPRECATE_EOF) - packet_sender->sendPacket(OK_Packet(0xfe, context->mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true); + packet_sender->sendPacket(OK_Packet(0xfe, context->mysql.client_capabilities, affected_rows, 0, 0, "", human_readable_info), true); else packet_sender->sendPacket(EOF_Packet(0, 0), true); } diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index defadd0caec..9f797b32123 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -53,7 +53,7 @@ void MergingSortedTransform::onFinish() LOG_DEBUG(log, "Merge sorted {} blocks, {} rows in {} sec., {} rows/sec., {}/sec", merged_data.totalChunks(), merged_data.totalMergedRows(), seconds, merged_data.totalMergedRows() / seconds, - formatReadableSizeWithBinarySuffix(merged_data.totalAllocatedBytes() / seconds)); + ReadableSize(merged_data.totalAllocatedBytes() / seconds)); } } diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index fc8fd575c35..d8eff53f567 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -542,9 +542,9 @@ void AggregatingTransform::initGenerate() size_t rows = variants.sizeWithoutOverflowRow(); LOG_TRACE(log, "Aggregated. {} to {} rows (from {}) in {} sec. ({} rows/sec., {}/sec.)", - src_rows, rows, formatReadableSizeWithBinarySuffix(src_bytes), + src_rows, rows, ReadableSize(src_bytes), elapsed_seconds, src_rows / elapsed_seconds, - formatReadableSizeWithBinarySuffix(src_bytes / elapsed_seconds)); + ReadableSize(src_bytes / elapsed_seconds)); if (params->aggregator.hasTemporaryFiles()) { @@ -593,7 +593,7 @@ void AggregatingTransform::initGenerate() for (const auto & file : files.files) processors.emplace_back(std::make_unique(header, file->path())); - LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), formatReadableSizeWithBinarySuffix(files.sum_size_compressed), formatReadableSizeWithBinarySuffix(files.sum_size_uncompressed)); + LOG_TRACE(log, "Will merge {} temporary files of size {} compressed, {} uncompressed.", files.files.size(), ReadableSize(files.sum_size_compressed), ReadableSize(files.sum_size_uncompressed)); auto pipe = createMergingAggregatedMemoryEfficientPipe( header, params, files.files.size(), temporary_data_merge_threads); diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index 34eb833c411..279faaec8ac 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -267,7 +267,7 @@ void MergeSortingTransform::remerge() new_chunks.emplace_back(std::move(chunk)); } - LOG_DEBUG(log, "Memory usage is lowered from {} to {}", formatReadableSizeWithBinarySuffix(sum_bytes_in_blocks), formatReadableSizeWithBinarySuffix(new_sum_bytes_in_blocks)); + LOG_DEBUG(log, "Memory usage is lowered from {} to {}", ReadableSize(sum_bytes_in_blocks), ReadableSize(new_sum_bytes_in_blocks)); /// If the memory consumption was not lowered enough - we will not perform remerge anymore. 2 is a guess. if (new_sum_bytes_in_blocks * 2 > sum_bytes_in_blocks) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 175cf53ca93..215d5f493e5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2820,8 +2820,7 @@ inline ReservationPtr checkAndReturnReservation(UInt64 expected_size, Reservatio if (reservation) return reservation; - throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(expected_size) + ", not enough space", - ErrorCodes::NOT_ENOUGH_SPACE); + throw Exception(fmt::format("Cannot reserve {}, not enough space", ReadableSize(expected_size)), ErrorCodes::NOT_ENOUGH_SPACE); } } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index e36391f87ec..c83caae3aff 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -371,14 +371,13 @@ bool MergeTreeDataMergerMutator::selectAllPartsToMergeWithinPartition( ", {} required now (+{}% on overhead); suppressing similar warnings for the next hour", parts.front()->name, (*prev_it)->name, - formatReadableSizeWithBinarySuffix(available_disk_space), - formatReadableSizeWithBinarySuffix(sum_bytes), + ReadableSize(available_disk_space), + ReadableSize(sum_bytes), static_cast((DISK_USAGE_COEFFICIENT_TO_SELECT - 1.0) * 100)); } if (out_disable_reason) - *out_disable_reason = "Insufficient available disk space, required " + - formatReadableSizeWithDecimalSuffix(required_disk_space); + *out_disable_reason = fmt::format("Insufficient available disk space, required {}", ReadableSize(required_disk_space)); return false; } @@ -949,7 +948,15 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor /// Print overall profiling info. NOTE: it may duplicates previous messages { double elapsed_seconds = merge_entry->watch.elapsedSeconds(); - LOG_DEBUG(log, "Merge sorted {} rows, containing {} columns ({} merged, {} gathered) in {} sec., {} rows/sec., {}/sec.", merge_entry->rows_read, all_column_names.size(), merging_column_names.size(), gathering_column_names.size(), elapsed_seconds, merge_entry->rows_read / elapsed_seconds, formatReadableSizeWithBinarySuffix(merge_entry->bytes_read_uncompressed / elapsed_seconds)); + LOG_DEBUG(log, + "Merge sorted {} rows, containing {} columns ({} merged, {} gathered) in {} sec., {} rows/sec., {}/sec.", + merge_entry->rows_read, + all_column_names.size(), + merging_column_names.size(), + gathering_column_names.size(), + elapsed_seconds, + merge_entry->rows_read / elapsed_seconds, + ReadableSize(merge_entry->bytes_read_uncompressed / elapsed_seconds)); } if (merge_alg != MergeAlgorithm::Vertical) diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 4e564b512d6..e84ff418bc3 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -179,7 +179,7 @@ bool MergeTreePartsMover::selectPartsForMove( if (!parts_to_move.empty()) { - LOG_TRACE(log, "Selected {} parts to move according to storage policy rules and {} parts according to TTL rules, {} total", parts_to_move_by_policy_rules, parts_to_move_by_ttl_rules, formatReadableSizeWithBinarySuffix(parts_to_move_total_size_bytes)); + LOG_TRACE(log, "Selected {} parts to move according to storage policy rules and {} parts according to TTL rules, {} total", parts_to_move_by_policy_rules, parts_to_move_by_ttl_rules, ReadableSize(parts_to_move_total_size_bytes)); return true; } else diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index b266b65e19e..9ca1446ef64 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -169,7 +169,7 @@ void MergeTreeReadPool::profileFeedback(const ReadBufferFromFileBase::ProfileInf ProfileEvents::increment(ProfileEvents::SlowRead); LOG_DEBUG(log, "Slow read, event â„–{}: read {} bytes in {} sec., {}/s.", backoff_state.num_events, info.bytes_read, info.nanoseconds / 1e9, - formatReadableSizeWithBinarySuffix(throughput)); + ReadableSize(throughput)); if (backoff_state.num_events < backoff_settings.min_events) return; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c78f1f4f97a..9c8d1e9ac5c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1030,11 +1030,18 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( if (!ignore_max_size && sum_parts_size_in_bytes > max_source_parts_size) { - String reason = "Not executing log entry " + entry.typeToString() + " for part " + entry.new_part_name - + " because source parts size (" + formatReadableSizeWithBinarySuffix(sum_parts_size_in_bytes) - + ") is greater than the current maximum (" + formatReadableSizeWithBinarySuffix(max_source_parts_size) + ")."; - LOG_DEBUG(log, reason); - out_postpone_reason = reason; + const char * format_str = "Not executing log entry {} for part {}" + " because source parts size ({}) is greater than the current maximum ({})."; + + LOG_DEBUG(log, format_str, + entry.typeToString(), entry.new_part_name, + ReadableSize(sum_parts_size_in_bytes), ReadableSize(max_source_parts_size)); + + /// Copy-paste of above because we need structured logging (instead of already formatted message). + out_postpone_reason = fmt::format(format_str, + entry.typeToString(), entry.new_part_name, + ReadableSize(sum_parts_size_in_bytes), ReadableSize(max_source_parts_size)); + return false; } } diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 4682b873dc5..a1ba39ef158 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -192,7 +192,7 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) /// TODO Add speed, compressed bytes, data volume in memory, compression ratio ... Generalize all statistics logging in project. LOG_INFO(&Logger::get("StorageSetOrJoinBase"), "Loaded from backup file {}. {} rows, {}. State has {} unique rows.", - file_path, backup_stream.getProfileInfo().rows, formatReadableSizeWithBinarySuffix(backup_stream.getProfileInfo().bytes), getSize()); + file_path, backup_stream.getProfileInfo().rows, ReadableSize(backup_stream.getProfileInfo().bytes), getSize()); } From 25f941020b578f2e9d479ae7df02d0148619fdb1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 00:57:37 +0300 Subject: [PATCH 124/150] Remove namespace pollution --- base/common/ErrorHandlers.h | 2 +- base/common/logger_useful.h | 25 +++++++------------ base/daemon/BaseDaemon.cpp | 10 ++++---- programs/local/LocalServer.cpp | 2 +- programs/odbc-bridge/ODBCBlockInputStream.cpp | 2 +- .../odbc-bridge/ODBCBlockOutputStream.cpp | 2 +- programs/server/Server.cpp | 4 +-- src/Access/AllowedClientHosts.cpp | 4 +-- src/Client/Connection.h | 6 ++--- src/Client/ConnectionPool.h | 2 +- src/Client/ConnectionPoolWithFailover.cpp | 2 +- src/Client/TimeoutSetter.cpp | 2 +- src/Common/Config/ConfigProcessor.cpp | 8 +++--- src/Common/Config/ConfigProcessor.h | 2 +- src/Common/Config/ConfigReloader.h | 2 +- src/Common/DNSResolver.cpp | 2 +- src/Common/Exception.cpp | 2 +- src/Common/FileChecker.h | 2 +- src/Common/LRUCache.h | 4 +-- src/Common/MemoryTracker.cpp | 4 +-- src/Common/PoolBase.h | 4 +-- src/Common/PoolWithFailoverBase.h | 4 +-- src/Common/QueryProfiler.cpp | 2 +- src/Common/SensitiveDataMasker.cpp | 2 +- src/Common/StatusFile.cpp | 8 +++--- src/Common/ZooKeeper/Increment.h | 2 +- src/Common/ZooKeeper/LeaderElection.h | 4 +-- src/Common/ZooKeeper/Lock.h | 4 +-- src/Common/ZooKeeper/ZooKeeper.cpp | 2 +- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeperHolder.h | 2 +- .../tests/zkutil_expiration_test.cpp | 4 +-- src/Common/tests/gtest_log.cpp | 2 +- src/Core/BackgroundSchedulePool.cpp | 6 ++--- src/Core/MySQLProtocol.h | 4 +-- src/Core/SettingsCollection.cpp | 2 +- src/DataStreams/AggregatingBlockInputStream.h | 2 +- .../CollapsingFinalBlockInputStream.h | 2 +- src/DataStreams/ColumnGathererStream.cpp | 2 +- .../CreatingSetsBlockInputStream.h | 2 +- .../MergeSortingBlockInputStream.h | 2 +- ...regatedMemoryEfficientBlockInputStream.cpp | 2 +- ...ggregatedMemoryEfficientBlockInputStream.h | 2 +- .../MergingSortedBlockInputStream.cpp | 2 +- .../ParallelAggregatingBlockInputStream.h | 2 +- src/DataStreams/ParallelInputsProcessor.h | 2 +- src/DataStreams/RemoteBlockInputStream.h | 2 +- src/DataStreams/TTLBlockInputStream.cpp | 2 +- src/DataStreams/TTLBlockInputStream.h | 2 +- src/DataStreams/UnionBlockInputStream.h | 2 +- .../tests/collapsing_sorted_stream.cpp | 4 +-- src/Databases/DatabaseDictionary.cpp | 2 +- src/Databases/DatabasesCommon.cpp | 2 +- src/Dictionaries/CacheDictionary.cpp | 2 +- src/Dictionaries/CacheDictionary.h | 4 +-- .../Embedded/RegionsHierarchies.cpp | 2 +- .../Embedded/RegionsHierarchy.cpp | 2 +- src/Dictionaries/Embedded/RegionsNames.cpp | 2 +- .../ExecutableDictionarySource.cpp | 4 +-- src/Dictionaries/HTTPDictionarySource.cpp | 4 +-- src/Dictionaries/LibraryDictionarySource.cpp | 4 +-- .../LibraryDictionarySourceExternal.cpp | 2 +- src/Dictionaries/MySQLDictionarySource.cpp | 4 +-- src/Dictionaries/TrieDictionary.h | 2 +- src/Dictionaries/XDBCDictionarySource.cpp | 4 +-- src/Disks/DiskLocal.cpp | 8 +++--- src/Disks/DiskSelector.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 16 ++++++------ src/Disks/S3/ProxyListConfiguration.cpp | 2 +- src/Disks/S3/ProxyResolverConfiguration.cpp | 4 +-- src/Disks/S3/registerDiskS3.cpp | 4 +-- src/Disks/StoragePolicy.cpp | 2 +- src/Disks/VolumeJBOD.cpp | 2 +- src/IO/HTTPCommon.cpp | 2 +- src/IO/ReadBufferFromS3.h | 2 +- src/IO/ReadWriteBufferFromHTTP.h | 2 +- src/IO/S3Common.cpp | 19 +++++++------- src/IO/WriteBufferFromHTTP.cpp | 2 +- src/IO/WriteBufferFromS3.h | 2 +- src/Interpreters/Aggregator.cpp | 2 +- src/Interpreters/Aggregator.h | 2 +- .../ClusterProxy/SelectStreamFactory.cpp | 6 ++--- src/Interpreters/Context.cpp | 2 +- src/Interpreters/DDLWorker.cpp | 6 ++--- src/Interpreters/DDLWorker.h | 2 +- src/Interpreters/EmbeddedDictionaries.cpp | 2 +- .../ExternalDictionariesLoader.cpp | 2 +- src/Interpreters/ExternalLoader.cpp | 10 ++++---- src/Interpreters/ExternalLoader.h | 2 +- src/Interpreters/ExternalModelsLoader.cpp | 2 +- src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/PartLog.cpp | 2 +- src/Interpreters/ProcessList.cpp | 6 ++--- src/Interpreters/Set.h | 4 +-- src/Interpreters/SystemLog.cpp | 2 +- src/Interpreters/SystemLog.h | 4 +-- src/Interpreters/executeQuery.cpp | 12 ++++----- src/Interpreters/tests/select_query.cpp | 4 +-- src/Processors/Formats/IRowInputFormat.cpp | 2 +- .../Formats/Impl/AvroRowInputFormat.cpp | 4 +-- .../Algorithms/CollapsingSortedAlgorithm.cpp | 2 +- .../Algorithms/CollapsingSortedAlgorithm.h | 4 +-- src/Processors/Merges/Algorithms/RowRef.h | 4 +-- .../Merges/CollapsingSortedTransform.h | 2 +- .../Merges/MergingSortedTransform.cpp | 2 +- .../Transforms/AggregatingTransform.h | 2 +- .../Transforms/CreatingSetsTransform.h | 2 +- .../Transforms/MergeSortingTransform.cpp | 4 +-- .../Transforms/MergeSortingTransform.h | 2 +- .../Transforms/MergingAggregatedTransform.h | 2 +- .../tests/processors_test_aggregation.cpp | 4 +-- ...rocessors_test_merge_sorting_transform.cpp | 4 +-- src/Server/HTTPHandler.cpp | 2 +- src/Server/HTTPHandlerFactory.cpp | 2 +- src/Server/HTTPHandlerFactory.h | 2 +- src/Server/MySQLHandlerFactory.cpp | 2 +- src/Server/ReplicasStatusHandler.cpp | 2 +- src/Server/TCPHandlerFactory.h | 2 +- src/Storages/Distributed/DirectoryMonitor.cpp | 8 +++--- src/Storages/Distributed/DirectoryMonitor.h | 4 +-- .../DistributedBlockOutputStream.cpp | 2 +- src/Storages/Kafka/StorageKafka.cpp | 2 +- .../MergeTree/BackgroundProcessingPool.cpp | 2 +- src/Storages/MergeTree/DataPartsExchange.h | 8 +++--- .../MergeTree/EphemeralLockInZooKeeper.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataMergerMutator.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.h | 4 +-- src/Storages/MergeTree/MergeTreePartsMover.h | 2 +- src/Storages/MergeTree/MergeTreeReadPool.h | 2 +- .../MergeTreeReverseSelectProcessor.h | 2 +- .../MergeTree/MergeTreeSelectProcessor.h | 2 +- .../MergeTree/MergeTreeSequentialSource.h | 2 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 2 +- .../ReplicatedMergeTreeBlockOutputStream.h | 2 +- .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- .../ReplicatedMergeTreeCleanupThread.h | 2 +- .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- .../ReplicatedMergeTreePartCheckThread.h | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.h | 2 +- .../ReplicatedMergeTreeRestartingThread.cpp | 2 +- .../ReplicatedMergeTreeRestartingThread.h | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageDistributed.h | 2 +- src/Storages/StorageFile.h | 2 +- src/Storages/StorageHDFS.h | 2 +- src/Storages/StorageSet.cpp | 2 +- src/Storages/StorageStripeLog.cpp | 2 +- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTinyLog.cpp | 2 +- src/Storages/StorageTinyLog.h | 2 +- utils/zookeeper-cli/zookeeper-cli.cpp | 4 +-- 160 files changed, 253 insertions(+), 259 deletions(-) diff --git a/base/common/ErrorHandlers.h b/base/common/ErrorHandlers.h index b35cdcab93d..67194ee043b 100644 --- a/base/common/ErrorHandlers.h +++ b/base/common/ErrorHandlers.h @@ -28,7 +28,7 @@ public: void exception() override { logException(); } private: - Logger * log = &Logger::get("ServerErrorHandler"); + Poco::Logger * log = &Poco::Logger::get("ServerErrorHandler"); void logException() { diff --git a/base/common/logger_useful.h b/base/common/logger_useful.h index ad56ef7e857..ec3933cf121 100644 --- a/base/common/logger_useful.h +++ b/base/common/logger_useful.h @@ -9,13 +9,6 @@ #include -/// TODO Remove this. -using Poco::Logger; -using Poco::Message; -using DB::LogsLevel; -using DB::CurrentThread; - - namespace { template constexpr size_t numArgs(Ts &&...) { return sizeof...(Ts); } @@ -31,8 +24,8 @@ namespace #define LOG_IMPL(logger, priority, PRIORITY, ...) do \ { \ - const bool is_clients_log = (CurrentThread::getGroup() != nullptr) && \ - (CurrentThread::getGroup()->client_logs_level >= (priority)); \ + const bool is_clients_log = (DB::CurrentThread::getGroup() != nullptr) && \ + (DB::CurrentThread::getGroup()->client_logs_level >= (priority)); \ if ((logger)->is((PRIORITY)) || is_clients_log) \ { \ std::string formatted_message = numArgs(__VA_ARGS__) > 1 ? fmt::format(__VA_ARGS__) : firstArg(__VA_ARGS__); \ @@ -42,7 +35,7 @@ namespace file_function += __FILE__; \ file_function += "; "; \ file_function += __PRETTY_FUNCTION__; \ - Message poco_message((logger)->name(), formatted_message, \ + Poco::Message poco_message((logger)->name(), formatted_message, \ (PRIORITY), file_function.c_str(), __LINE__); \ channel->log(poco_message); \ } \ @@ -50,9 +43,9 @@ namespace } while (false) -#define LOG_TRACE(logger, ...) LOG_IMPL(logger, LogsLevel::trace, Message::PRIO_TRACE, __VA_ARGS__) -#define LOG_DEBUG(logger, ...) LOG_IMPL(logger, LogsLevel::debug, Message::PRIO_DEBUG, __VA_ARGS__) -#define LOG_INFO(logger, ...) LOG_IMPL(logger, LogsLevel::information, Message::PRIO_INFORMATION, __VA_ARGS__) -#define LOG_WARNING(logger, ...) LOG_IMPL(logger, LogsLevel::warning, Message::PRIO_WARNING, __VA_ARGS__) -#define LOG_ERROR(logger, ...) LOG_IMPL(logger, LogsLevel::error, Message::PRIO_ERROR, __VA_ARGS__) -#define LOG_FATAL(logger, ...) LOG_IMPL(logger, LogsLevel::error, Message::PRIO_FATAL, __VA_ARGS__) +#define LOG_TRACE(logger, ...) LOG_IMPL(logger, DB::LogsLevel::trace, Poco::Message::PRIO_TRACE, __VA_ARGS__) +#define LOG_DEBUG(logger, ...) LOG_IMPL(logger, DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG, __VA_ARGS__) +#define LOG_INFO(logger, ...) LOG_IMPL(logger, DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION, __VA_ARGS__) +#define LOG_WARNING(logger, ...) LOG_IMPL(logger, DB::LogsLevel::warning, Poco::Message::PRIO_WARNING, __VA_ARGS__) +#define LOG_ERROR(logger, ...) LOG_IMPL(logger, DB::LogsLevel::error, Poco::Message::PRIO_ERROR, __VA_ARGS__) +#define LOG_FATAL(logger, ...) LOG_IMPL(logger, DB::LogsLevel::error, Poco::Message::PRIO_FATAL, __VA_ARGS__) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 10c7173d5b1..1ab140d2595 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -124,7 +124,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context) const ucontext_t signal_context = *reinterpret_cast(context); const StackTrace stack_trace(signal_context); - StringRef query_id = CurrentThread::getQueryId(); /// This is signal safe. + StringRef query_id = DB::CurrentThread::getQueryId(); /// This is signal safe. query_id.size = std::min(query_id.size, max_query_id_size); DB::writeBinary(sig, out); @@ -162,7 +162,7 @@ public: }; explicit SignalListener(BaseDaemon & daemon_) - : log(&Logger::get("BaseDaemon")) + : log(&Poco::Logger::get("BaseDaemon")) , daemon(daemon_) { } @@ -231,7 +231,7 @@ public: } private: - Logger * log; + Poco::Logger * log; BaseDaemon & daemon; void onTerminate(const std::string & message, UInt32 thread_num) const @@ -715,7 +715,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() void BaseDaemon::logRevision() const { - Logger::root().information("Starting " + std::string{VERSION_FULL} + Poco::Logger::root().information("Starting " + std::string{VERSION_FULL} + " with revision " + std::to_string(ClickHouseRevision::get()) + ", PID " + std::to_string(getpid())); } @@ -732,7 +732,7 @@ void BaseDaemon::handleNotification(Poco::TaskFailedNotification *_tfn) { task_failed = true; Poco::AutoPtr fn(_tfn); - Logger *lg = &(logger()); + Poco::Logger * lg = &(logger()); LOG_ERROR(lg, "Task '{}' failed. Daemon is shutting down. Reason - {}", fn->task()->name(), fn->reason().displayText()); ServerApplication::terminate(); } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index eb78c049825..8ce03ac0867 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -135,7 +135,7 @@ static void attachSystemTables(const Context & context) int LocalServer::main(const std::vector & /*args*/) try { - Logger * log = &logger(); + Poco::Logger * log = &logger(); ThreadStatus thread_status; UseSSL use_ssl; diff --git a/programs/odbc-bridge/ODBCBlockInputStream.cpp b/programs/odbc-bridge/ODBCBlockInputStream.cpp index 6ada6dd3a2d..1316ff8f4c6 100644 --- a/programs/odbc-bridge/ODBCBlockInputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockInputStream.cpp @@ -25,7 +25,7 @@ ODBCBlockInputStream::ODBCBlockInputStream( , result{statement} , iterator{result.begin()} , max_block_size{max_block_size_} - , log(&Logger::get("ODBCBlockInputStream")) + , log(&Poco::Logger::get("ODBCBlockInputStream")) { if (sample_block.columns() != result.columnCount()) throw Exception{"RecordSet contains " + toString(result.columnCount()) + " columns while " + toString(sample_block.columns()) diff --git a/programs/odbc-bridge/ODBCBlockOutputStream.cpp b/programs/odbc-bridge/ODBCBlockOutputStream.cpp index ab24c008e40..b5bffc58c55 100644 --- a/programs/odbc-bridge/ODBCBlockOutputStream.cpp +++ b/programs/odbc-bridge/ODBCBlockOutputStream.cpp @@ -94,7 +94,7 @@ ODBCBlockOutputStream::ODBCBlockOutputStream(Poco::Data::Session && session_, , table_name(remote_table_name_) , sample_block(sample_block_) , quoting(quoting_) - , log(&Logger::get("ODBCBlockOutputStream")) + , log(&Poco::Logger::get("ODBCBlockOutputStream")) { description.init(sample_block); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 29096327a71..e587e134075 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -89,7 +89,7 @@ namespace CurrentMetrics namespace { -void setupTmpPath(Logger * log, const std::string & path) +void setupTmpPath(Poco::Logger * log, const std::string & path) { LOG_DEBUG(log, "Setting up {} to store temporary data in it", path); @@ -212,7 +212,7 @@ void Server::defineOptions(Poco::Util::OptionSet & options) int Server::main(const std::vector & /*args*/) { - Logger * log = &logger(); + Poco::Logger * log = &logger(); UseSSL use_ssl; ThreadStatus thread_status; diff --git a/src/Access/AllowedClientHosts.cpp b/src/Access/AllowedClientHosts.cpp index 9c8a7cc12f5..82372fd8b14 100644 --- a/src/Access/AllowedClientHosts.cpp +++ b/src/Access/AllowedClientHosts.cpp @@ -309,7 +309,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const throw; /// Try to ignore DNS errors: if host cannot be resolved, skip it and try next. LOG_WARNING( - &Logger::get("AddressPatterns"), + &Poco::Logger::get("AddressPatterns"), "Failed to check if the allowed client hosts contain address {}. {}, code = {}", client_address.toString(), e.displayText(), e.code()); return false; @@ -342,7 +342,7 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const throw; /// Try to ignore DNS errors: if host cannot be resolved, skip it and try next. LOG_WARNING( - &Logger::get("AddressPatterns"), + &Poco::Logger::get("AddressPatterns"), "Failed to check if the allowed client hosts contain address {}. {}, code = {}", client_address.toString(), e.displayText(), e.code()); return false; diff --git a/src/Client/Connection.h b/src/Client/Connection.h index e056a4323df..f93fa7871e5 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -249,16 +249,16 @@ private: { } - Logger * get() + Poco::Logger * get() { if (!log) - log = &Logger::get("Connection (" + parent.getDescription() + ")"); + log = &Poco::Logger::get("Connection (" + parent.getDescription() + ")"); return log; } private: - std::atomic log; + std::atomic log; Connection & parent; }; diff --git a/src/Client/ConnectionPool.h b/src/Client/ConnectionPool.h index be52234b904..bbdcae894e7 100644 --- a/src/Client/ConnectionPool.h +++ b/src/Client/ConnectionPool.h @@ -56,7 +56,7 @@ public: Protocol::Compression compression_ = Protocol::Compression::Enable, Protocol::Secure secure_ = Protocol::Secure::Disable) : Base(max_connections_, - &Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")), + &Poco::Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")), host(host_), port(port_), default_database(default_database_), diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 9933cc6b555..713bb33342f 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -35,7 +35,7 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover( LoadBalancing load_balancing, time_t decrease_error_period_, size_t max_error_cap_) - : Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Logger::get("ConnectionPoolWithFailover")) + : Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Poco::Logger::get("ConnectionPoolWithFailover")) , default_load_balancing(load_balancing) { const std::string & local_hostname = getFQDNOrHostName(); diff --git a/src/Client/TimeoutSetter.cpp b/src/Client/TimeoutSetter.cpp index 10cc9fbffef..a512485407e 100644 --- a/src/Client/TimeoutSetter.cpp +++ b/src/Client/TimeoutSetter.cpp @@ -35,7 +35,7 @@ TimeoutSetter::~TimeoutSetter() catch (std::exception & e) { // Sometimes catched on macos - LOG_ERROR(&Logger::get("Client"), "TimeoutSetter: Can't reset timeouts: {}", e.what()); + LOG_ERROR(&Poco::Logger::get("Client"), "TimeoutSetter: Can't reset timeouts: {}", e.what()); } } } diff --git a/src/Common/Config/ConfigProcessor.cpp b/src/Common/Config/ConfigProcessor.cpp index 7eb7e13a93e..bf65b7028cc 100644 --- a/src/Common/Config/ConfigProcessor.cpp +++ b/src/Common/Config/ConfigProcessor.cpp @@ -66,21 +66,21 @@ ConfigProcessor::ConfigProcessor( , name_pool(new Poco::XML::NamePool(65521)) , dom_parser(name_pool) { - if (log_to_console && !Logger::has("ConfigProcessor")) + if (log_to_console && !Poco::Logger::has("ConfigProcessor")) { channel_ptr = new Poco::ConsoleChannel; - log = &Logger::create("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE); + log = &Poco::Logger::create("ConfigProcessor", channel_ptr.get(), Poco::Message::PRIO_TRACE); } else { - log = &Logger::get("ConfigProcessor"); + log = &Poco::Logger::get("ConfigProcessor"); } } ConfigProcessor::~ConfigProcessor() { if (channel_ptr) /// This means we have created a new console logger in the constructor. - Logger::destroy("ConfigProcessor"); + Poco::Logger::destroy("ConfigProcessor"); } diff --git a/src/Common/Config/ConfigProcessor.h b/src/Common/Config/ConfigProcessor.h index b6f772f8c16..5a942e73bdb 100644 --- a/src/Common/Config/ConfigProcessor.h +++ b/src/Common/Config/ConfigProcessor.h @@ -116,7 +116,7 @@ private: bool throw_on_bad_incl; - Logger * log; + Poco::Logger * log; Poco::AutoPtr channel_ptr; Substitutions substitutions; diff --git a/src/Common/Config/ConfigReloader.h b/src/Common/Config/ConfigReloader.h index c0904422b39..553589fbd37 100644 --- a/src/Common/Config/ConfigReloader.h +++ b/src/Common/Config/ConfigReloader.h @@ -69,7 +69,7 @@ private: static constexpr auto reload_interval = std::chrono::seconds(2); - Poco::Logger * log = &Logger::get("ConfigReloader"); + Poco::Logger * log = &Poco::Logger::get("ConfigReloader"); std::string path; std::string include_from_path; diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 1d2d89f477c..c5b4fd0c585 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -202,7 +202,7 @@ bool DNSResolver::updateCache() } if (!lost_hosts.empty()) - LOG_INFO(&Logger::get("DNSResolver"), "Cached hosts not found: {}", lost_hosts); + LOG_INFO(&Poco::Logger::get("DNSResolver"), "Cached hosts not found: {}", lost_hosts); return updated; } diff --git a/src/Common/Exception.cpp b/src/Common/Exception.cpp index 0233fb00a43..647e0372a4b 100644 --- a/src/Common/Exception.cpp +++ b/src/Common/Exception.cpp @@ -118,7 +118,7 @@ void throwFromErrnoWithPath(const std::string & s, const std::string & path, int void tryLogCurrentException(const char * log_name, const std::string & start_of_message) { - tryLogCurrentException(&Logger::get(log_name), start_of_message); + tryLogCurrentException(&Poco::Logger::get(log_name), start_of_message); } void tryLogCurrentException(Poco::Logger * logger, const std::string & start_of_message) diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index c4a919e9ec1..83db397e78c 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -37,7 +37,7 @@ private: Map map; bool initialized = false; - Logger * log = &Logger::get("FileChecker"); + Poco::Logger * log = &Poco::Logger::get("FileChecker"); }; } diff --git a/src/Common/LRUCache.h b/src/Common/LRUCache.h index 5bcfc8fc2db..76aa9705a91 100644 --- a/src/Common/LRUCache.h +++ b/src/Common/LRUCache.h @@ -306,7 +306,7 @@ private: auto it = cells.find(key); if (it == cells.end()) { - LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); + LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); } @@ -324,7 +324,7 @@ private: if (current_size > (1ull << 63)) { - LOG_ERROR(&Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); + LOG_ERROR(&Poco::Logger::get("LRUCache"), "LRUCache became inconsistent. There must be a bug in it."); abort(); } } diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 54e991da41f..03bd8be94f3 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -50,13 +50,13 @@ MemoryTracker::~MemoryTracker() void MemoryTracker::logPeakMemoryUsage() const { const auto * description = description_ptr.load(std::memory_order_relaxed); - LOG_DEBUG(&Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak)); + LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "Peak memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(peak)); } void MemoryTracker::logMemoryUsage(Int64 current) const { const auto * description = description_ptr.load(std::memory_order_relaxed); - LOG_DEBUG(&Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current)); + LOG_DEBUG(&Poco::Logger::get("MemoryTracker"), "Current memory usage{}: {}.", (description ? " " + std::string(description) : ""), ReadableSize(current)); } diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index 201f9ec1d00..43f4fbff9fe 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -152,9 +152,9 @@ private: protected: - Logger * log; + Poco::Logger * log; - PoolBase(unsigned max_items_, Logger * log_) + PoolBase(unsigned max_items_, Poco::Logger * log_) : max_items(max_items_), log(log_) { items.reserve(max_items); diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index e9f3d7d5d23..4d9e52364b9 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -57,7 +57,7 @@ public: NestedPools nested_pools_, time_t decrease_error_period_, size_t max_error_cap_, - Logger * log_) + Poco::Logger * log_) : nested_pools(std::move(nested_pools_)) , decrease_error_period(decrease_error_period_) , max_error_cap(max_error_cap_) @@ -134,7 +134,7 @@ protected: /// The time when error counts were last decreased. time_t last_error_decrease_time = 0; - Logger * log; + Poco::Logger * log; }; template diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index cd68edd3ba1..a8b7d51a260 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -79,7 +79,7 @@ namespace ErrorCodes template QueryProfilerBase::QueryProfilerBase(const UInt64 thread_id, const int clock_type, UInt32 period, const int pause_signal_) - : log(&Logger::get("QueryProfiler")) + : log(&Poco::Logger::get("QueryProfiler")) , pause_signal(pause_signal_) { #if USE_UNWIND diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 0d8e155d44a..d3d9991bc90 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -102,7 +102,7 @@ SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration { Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix, keys); - Logger * logger = &Logger::get("SensitiveDataMaskerConfigRead"); + Poco::Logger * logger = &Poco::Logger::get("SensitiveDataMaskerConfigRead"); std::set used_names; diff --git a/src/Common/StatusFile.cpp b/src/Common/StatusFile.cpp index 79e34d244bc..758f500e9d2 100644 --- a/src/Common/StatusFile.cpp +++ b/src/Common/StatusFile.cpp @@ -43,9 +43,9 @@ StatusFile::StatusFile(const std::string & path_) } if (!contents.empty()) - LOG_INFO(&Logger::get("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents); + LOG_INFO(&Poco::Logger::get("StatusFile"), "Status file {} already exists - unclean restart. Contents:\n{}", path, contents); else - LOG_INFO(&Logger::get("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path); + LOG_INFO(&Poco::Logger::get("StatusFile"), "Status file {} already exists and is empty - probably unclean hardware restart.", path); } fd = ::open(path.c_str(), O_WRONLY | O_CREAT | O_CLOEXEC, 0666); @@ -90,10 +90,10 @@ StatusFile::StatusFile(const std::string & path_) StatusFile::~StatusFile() { if (0 != close(fd)) - LOG_ERROR(&Logger::get("StatusFile"), "Cannot close file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE)); + LOG_ERROR(&Poco::Logger::get("StatusFile"), "Cannot close file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE)); if (0 != unlink(path.c_str())) - LOG_ERROR(&Logger::get("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE)); + LOG_ERROR(&Poco::Logger::get("StatusFile"), "Cannot unlink file {}, {}", path, errnoToString(ErrorCodes::CANNOT_CLOSE_FILE)); } } diff --git a/src/Common/ZooKeeper/Increment.h b/src/Common/ZooKeeper/Increment.h index 26128480635..fa5f550ca9b 100644 --- a/src/Common/ZooKeeper/Increment.h +++ b/src/Common/ZooKeeper/Increment.h @@ -43,7 +43,7 @@ public: private: zkutil::ZooKeeperHolderPtr zookeeper_holder; std::string path; - Logger * log = &Logger::get("zkutil::Increment"); + Poco::Logger * log = &Poco::Logger::get("zkutil::Increment"); }; } diff --git a/src/Common/ZooKeeper/LeaderElection.h b/src/Common/ZooKeeper/LeaderElection.h index fe10a3bf00e..e3b97e7f8ca 100644 --- a/src/Common/ZooKeeper/LeaderElection.h +++ b/src/Common/ZooKeeper/LeaderElection.h @@ -39,7 +39,7 @@ public: LeaderElection(DB::BackgroundSchedulePool & pool_, const std::string & path_, ZooKeeper & zookeeper_, LeadershipHandler handler_, const std::string & identifier_ = "") : pool(pool_), path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_) , log_name("LeaderElection (" + path + ")") - , log(&Logger::get(log_name)) + , log(&Poco::Logger::get(log_name)) { task = pool.createTask(log_name, [this] { threadFunction(); }); createNode(); @@ -67,7 +67,7 @@ private: LeadershipHandler handler; std::string identifier; std::string log_name; - Logger * log; + Poco::Logger * log; EphemeralNodeHolderPtr node; std::string node_name; diff --git a/src/Common/ZooKeeper/Lock.h b/src/Common/ZooKeeper/Lock.h index 683470cf5a5..67116124b4a 100644 --- a/src/Common/ZooKeeper/Lock.h +++ b/src/Common/ZooKeeper/Lock.h @@ -21,7 +21,7 @@ namespace zkutil zookeeper_holder(zookeeper_holder_), lock_path(lock_prefix_ + "/" + lock_name_), lock_message(lock_message_), - log(&Logger::get("zkutil::Lock")) + log(&Poco::Logger::get("zkutil::Lock")) { auto zookeeper = zookeeper_holder->getZooKeeper(); if (create_parent_path_) @@ -72,7 +72,7 @@ namespace zkutil std::string lock_path; std::string lock_message; - Logger * log; + Poco::Logger * log; }; } diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 6e0cc22a952..115518e2bf9 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -48,7 +48,7 @@ static void check(int32_t code, const std::string & path) void ZooKeeper::init(const std::string & implementation_, const std::string & hosts_, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_) { - log = &Logger::get("ZooKeeper"); + log = &Poco::Logger::get("ZooKeeper"); hosts = hosts_; identity = identity_; session_timeout_ms = session_timeout_ms_; diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index e8ab06c2182..3bf9ad3c100 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -269,7 +269,7 @@ private: std::mutex mutex; - Logger * log = nullptr; + Poco::Logger * log = nullptr; }; diff --git a/src/Common/ZooKeeper/ZooKeeperHolder.h b/src/Common/ZooKeeper/ZooKeeperHolder.h index d5792b8fde6..74a0a7fbc34 100644 --- a/src/Common/ZooKeeper/ZooKeeperHolder.h +++ b/src/Common/ZooKeeper/ZooKeeperHolder.h @@ -70,7 +70,7 @@ private: mutable std::mutex mutex; ZooKeeper::Ptr ptr; - Logger * log = &Logger::get("ZooKeeperHolder"); + Poco::Logger * log = &Poco::Logger::get("ZooKeeperHolder"); static std::string nullptr_exception_message; }; diff --git a/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp b/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp index 1c74985d5f9..d245428db8e 100644 --- a/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp +++ b/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp @@ -20,8 +20,8 @@ int main(int argc, char ** argv) } Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); zkutil::ZooKeeper zk(argv[1]); std::string unused; diff --git a/src/Common/tests/gtest_log.cpp b/src/Common/tests/gtest_log.cpp index a1e532f92e6..9f4ef41f642 100644 --- a/src/Common/tests/gtest_log.cpp +++ b/src/Common/tests/gtest_log.cpp @@ -12,7 +12,7 @@ TEST(Logger, Log) { Poco::Logger::root().setLevel("none"); Poco::Logger::root().setChannel(Poco::AutoPtr(new Poco::NullChannel())); - Logger * log = &Logger::get("Log"); + Poco::Logger * log = &Poco::Logger::get("Log"); /// This test checks that we don't pass this string to fmtlib, because it is the only argument. EXPECT_NO_THROW(LOG_INFO(log, "Hello {} World")); diff --git a/src/Core/BackgroundSchedulePool.cpp b/src/Core/BackgroundSchedulePool.cpp index cc9ee23c9b8..a72838c92e9 100644 --- a/src/Core/BackgroundSchedulePool.cpp +++ b/src/Core/BackgroundSchedulePool.cpp @@ -111,7 +111,7 @@ void BackgroundSchedulePoolTaskInfo::execute() static const int32_t slow_execution_threshold_ms = 200; if (milliseconds >= slow_execution_threshold_ms) - LOG_TRACE(&Logger::get(log_name), "Execution took {} ms.", milliseconds); + LOG_TRACE(&Poco::Logger::get(log_name), "Execution took {} ms.", milliseconds); { std::lock_guard lock_schedule(schedule_mutex); @@ -156,7 +156,7 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size_, CurrentMetrics::Met , memory_metric(memory_metric_) , thread_name(thread_name_) { - LOG_INFO(&Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size); + LOG_INFO(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Create BackgroundSchedulePool with {} threads", size); threads.resize(size); for (auto & thread : threads) @@ -179,7 +179,7 @@ BackgroundSchedulePool::~BackgroundSchedulePool() queue.wakeUpAll(); delayed_thread.join(); - LOG_TRACE(&Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish."); + LOG_TRACE(&Poco::Logger::get("BackgroundSchedulePool/" + thread_name), "Waiting for threads to finish."); for (auto & thread : threads) thread.join(); } diff --git a/src/Core/MySQLProtocol.h b/src/Core/MySQLProtocol.h index 2fd15f78c05..ee53276ee1b 100644 --- a/src/Core/MySQLProtocol.h +++ b/src/Core/MySQLProtocol.h @@ -994,7 +994,7 @@ private: class Sha256Password : public IPlugin { public: - Sha256Password(RSA & public_key_, RSA & private_key_, Logger * log_) + Sha256Password(RSA & public_key_, RSA & private_key_, Poco::Logger * log_) : public_key(public_key_) , private_key(private_key_) , log(log_) @@ -1130,7 +1130,7 @@ public: private: RSA & public_key; RSA & private_key; - Logger * log; + Poco::Logger * log; String scramble; }; #endif diff --git a/src/Core/SettingsCollection.cpp b/src/Core/SettingsCollection.cpp index 3b291406df9..324ad889a65 100644 --- a/src/Core/SettingsCollection.cpp +++ b/src/Core/SettingsCollection.cpp @@ -598,7 +598,7 @@ namespace details void SettingsCollectionUtils::warningNameNotFound(const StringRef & name) { - static auto * log = &Logger::get("Settings"); + static auto * log = &Poco::Logger::get("Settings"); LOG_WARNING(log, "Unknown setting {}, skipping", name); } diff --git a/src/DataStreams/AggregatingBlockInputStream.h b/src/DataStreams/AggregatingBlockInputStream.h index 5e993949b63..009a9704e4e 100644 --- a/src/DataStreams/AggregatingBlockInputStream.h +++ b/src/DataStreams/AggregatingBlockInputStream.h @@ -47,7 +47,7 @@ protected: /** From here we will get the completed blocks after the aggregation. */ std::unique_ptr impl; - Logger * log = &Logger::get("AggregatingBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("AggregatingBlockInputStream"); }; } diff --git a/src/DataStreams/CollapsingFinalBlockInputStream.h b/src/DataStreams/CollapsingFinalBlockInputStream.h index c237d5465c3..d090c53ddf9 100644 --- a/src/DataStreams/CollapsingFinalBlockInputStream.h +++ b/src/DataStreams/CollapsingFinalBlockInputStream.h @@ -168,7 +168,7 @@ private: const SortDescription description; String sign_column_name; - Logger * log = &Logger::get("CollapsingFinalBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("CollapsingFinalBlockInputStream"); bool first = true; diff --git a/src/DataStreams/ColumnGathererStream.cpp b/src/DataStreams/ColumnGathererStream.cpp index 4922a10b8a3..7a6dc7d88f0 100644 --- a/src/DataStreams/ColumnGathererStream.cpp +++ b/src/DataStreams/ColumnGathererStream.cpp @@ -21,7 +21,7 @@ ColumnGathererStream::ColumnGathererStream( const String & column_name_, const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_, size_t block_preferred_size_) : column_name(column_name_), sources(source_streams.size()), row_sources_buf(row_sources_buf_) - , block_preferred_size(block_preferred_size_), log(&Logger::get("ColumnGathererStream")) + , block_preferred_size(block_preferred_size_), log(&Poco::Logger::get("ColumnGathererStream")) { if (source_streams.empty()) throw Exception("There are no streams to gather", ErrorCodes::EMPTY_DATA_PASSED); diff --git a/src/DataStreams/CreatingSetsBlockInputStream.h b/src/DataStreams/CreatingSetsBlockInputStream.h index 4a4ffef29d6..178fa3f289f 100644 --- a/src/DataStreams/CreatingSetsBlockInputStream.h +++ b/src/DataStreams/CreatingSetsBlockInputStream.h @@ -44,7 +44,7 @@ private: size_t bytes_to_transfer = 0; using Logger = Poco::Logger; - Logger * log = &Logger::get("CreatingSetsBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("CreatingSetsBlockInputStream"); void createAll(); void createOne(SubqueryForSet & subquery); diff --git a/src/DataStreams/MergeSortingBlockInputStream.h b/src/DataStreams/MergeSortingBlockInputStream.h index ee03f202be0..c21c548ff24 100644 --- a/src/DataStreams/MergeSortingBlockInputStream.h +++ b/src/DataStreams/MergeSortingBlockInputStream.h @@ -104,7 +104,7 @@ private: String codec; size_t min_free_disk_space; - Logger * log = &Logger::get("MergeSortingBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("MergeSortingBlockInputStream"); Blocks blocks; size_t sum_rows_in_blocks = 0; diff --git a/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp b/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp index 9f514b6701d..56dcbda0487 100644 --- a/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp +++ b/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp @@ -555,7 +555,7 @@ MergingAggregatedMemoryEfficientBlockInputStream::BlocksToMerge MergingAggregate /// Not yet partitioned (splitted to buckets) block. Will partition it and place result to 'splitted_blocks'. if (input.block.info.bucket_num == -1 && input.block && input.splitted_blocks.empty()) { - LOG_TRACE(&Logger::get("MergingAggregatedMemoryEfficient"), "Having block without bucket: will split."); + LOG_TRACE(&Poco::Logger::get("MergingAggregatedMemoryEfficient"), "Having block without bucket: will split."); input.splitted_blocks = aggregator.convertBlockToTwoLevel(input.block); input.block = Block(); diff --git a/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h index 4a6eccfd2a4..f071ac42ee5 100644 --- a/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h +++ b/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -96,7 +96,7 @@ private: std::atomic has_overflows {false}; int current_bucket_num = -1; - Logger * log = &Logger::get("MergingAggregatedMemoryEfficientBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("MergingAggregatedMemoryEfficientBlockInputStream"); struct Input diff --git a/src/DataStreams/MergingSortedBlockInputStream.cpp b/src/DataStreams/MergingSortedBlockInputStream.cpp index 703ceff4c8f..434c89803b8 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -23,7 +23,7 @@ MergingSortedBlockInputStream::MergingSortedBlockInputStream( : description(std::move(description_)), max_block_size(max_block_size_), limit(limit_), quiet(quiet_) , source_blocks(inputs_.size()) , cursors(inputs_.size()), out_row_sources_buf(out_row_sources_buf_) - , log(&Logger::get("MergingSortedBlockInputStream")) + , log(&Poco::Logger::get("MergingSortedBlockInputStream")) { children.insert(children.end(), inputs_.begin(), inputs_.end()); header = children.at(0)->getHeader(); diff --git a/src/DataStreams/ParallelAggregatingBlockInputStream.h b/src/DataStreams/ParallelAggregatingBlockInputStream.h index 942c906b872..4b0a2e806fa 100644 --- a/src/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/src/DataStreams/ParallelAggregatingBlockInputStream.h @@ -60,7 +60,7 @@ private: std::atomic executed {false}; std::vector> temporary_inputs; - Logger * log = &Logger::get("ParallelAggregatingBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("ParallelAggregatingBlockInputStream"); ManyAggregatedDataVariants many_data; diff --git a/src/DataStreams/ParallelInputsProcessor.h b/src/DataStreams/ParallelInputsProcessor.h index 714174e6ac1..326506d28ca 100644 --- a/src/DataStreams/ParallelInputsProcessor.h +++ b/src/DataStreams/ParallelInputsProcessor.h @@ -359,7 +359,7 @@ private: /// Wait for the completion of all threads. std::atomic joined_threads { false }; - Logger * log = &Logger::get("ParallelInputsProcessor"); + Poco::Logger * log = &Poco::Logger::get("ParallelInputsProcessor"); }; diff --git a/src/DataStreams/RemoteBlockInputStream.h b/src/DataStreams/RemoteBlockInputStream.h index 66b1ebbb6c3..f6bac4155da 100644 --- a/src/DataStreams/RemoteBlockInputStream.h +++ b/src/DataStreams/RemoteBlockInputStream.h @@ -151,7 +151,7 @@ private: PoolMode pool_mode = PoolMode::GET_MANY; StorageID main_table = StorageID::createEmpty(); - Logger * log = &Logger::get("RemoteBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("RemoteBlockInputStream"); }; } diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index c6542763533..ca65ae520c6 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -28,7 +28,7 @@ TTLBlockInputStream::TTLBlockInputStream( , current_time(current_time_) , force(force_) , old_ttl_infos(data_part->ttl_infos) - , log(&Logger::get(storage.getLogName() + " (TTLBlockInputStream)")) + , log(&Poco::Logger::get(storage.getLogName() + " (TTLBlockInputStream)")) , date_lut(DateLUT::instance()) { children.push_back(input_); diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index c6ffa95cd75..060306f7d2d 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -52,7 +52,7 @@ private: NameSet empty_columns; size_t rows_removed = 0; - Logger * log; + Poco::Logger * log; const DateLUTImpl & date_lut; /// TODO rewrite defaults logic to evaluteMissingDefaults diff --git a/src/DataStreams/UnionBlockInputStream.h b/src/DataStreams/UnionBlockInputStream.h index c4e84e85845..ab667ac2e36 100644 --- a/src/DataStreams/UnionBlockInputStream.h +++ b/src/DataStreams/UnionBlockInputStream.h @@ -253,7 +253,7 @@ private: bool started = false; bool all_read = false; - Logger * log = &Logger::get("UnionBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("UnionBlockInputStream"); }; } diff --git a/src/DataStreams/tests/collapsing_sorted_stream.cpp b/src/DataStreams/tests/collapsing_sorted_stream.cpp index fd7dc11add6..e6d2167578b 100644 --- a/src/DataStreams/tests/collapsing_sorted_stream.cpp +++ b/src/DataStreams/tests/collapsing_sorted_stream.cpp @@ -20,8 +20,8 @@ try using namespace DB; Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); Block block1; diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index c4c74f8e70c..67d33d7bfd7 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -32,7 +32,7 @@ namespace DatabaseDictionary::DatabaseDictionary(const String & name_, const Context & global_context_) : IDatabase(name_) - , log(&Logger::get("DatabaseDictionary(" + database_name + ")")) + , log(&Poco::Logger::get("DatabaseDictionary(" + database_name + ")")) , global_context(global_context_.getGlobalContext()) { } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 2a2ca1841cf..47c54fae800 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes } DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, const Context & context) - : IDatabase(name_), log(&Logger::get(logger)), global_context(context.getGlobalContext()) + : IDatabase(name_), log(&Poco::Logger::get(logger)), global_context(context.getGlobalContext()) { } diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index fffeb796c56..99d2445807b 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -85,7 +85,7 @@ CacheDictionary::CacheDictionary( , update_queue_push_timeout_milliseconds(update_queue_push_timeout_milliseconds_) , query_wait_timeout_milliseconds(query_wait_timeout_milliseconds_) , max_threads_for_updates(max_threads_for_updates_) - , log(&Logger::get("ExternalDictionaries")) + , log(&Poco::Logger::get("ExternalDictionaries")) , size{roundUpToPowerOfTwoOrZero(std::max(size_, size_t(max_collision_length)))} , size_overlap_mask{this->size - 1} , cells{this->size} diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index bb103c61107..8109a6d706b 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -314,7 +314,7 @@ private: const size_t query_wait_timeout_milliseconds; const size_t max_threads_for_updates; - Logger * const log; + Poco::Logger * log; mutable std::shared_mutex rw_lock; @@ -356,7 +356,7 @@ private: * How the update goes: we basically have a method like get(keys)->values. Values are cached, so sometimes we * can return them from the cache. For values not in cache, we query them from the dictionary, and add to the * cache. The cache is lossy, so we can't expect it to store all the keys, and we store them separately. Normally, - * they would be passed as a return value of get(), but for Unknown Reasons the dictionaries use a baroque + * they would be passed as a return value of get(), but for Unknown Reasons the dictionaries use a baroque * interface where get() accepts two callback, one that it calls for found values, and one for not found. * * Now we make it even uglier by doing this from multiple threads. The missing values are retreived from the diff --git a/src/Dictionaries/Embedded/RegionsHierarchies.cpp b/src/Dictionaries/Embedded/RegionsHierarchies.cpp index 9fb51d72457..15e14db4664 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchies.cpp +++ b/src/Dictionaries/Embedded/RegionsHierarchies.cpp @@ -6,7 +6,7 @@ RegionsHierarchies::RegionsHierarchies(IRegionsHierarchiesDataProviderPtr data_provider) { - Logger * log = &Logger::get("RegionsHierarchies"); + Poco::Logger * log = &Poco::Logger::get("RegionsHierarchies"); LOG_DEBUG(log, "Adding default regions hierarchy"); data.emplace("", data_provider->getDefaultHierarchySource()); diff --git a/src/Dictionaries/Embedded/RegionsHierarchy.cpp b/src/Dictionaries/Embedded/RegionsHierarchy.cpp index dac1e948e0a..115ae30d93e 100644 --- a/src/Dictionaries/Embedded/RegionsHierarchy.cpp +++ b/src/Dictionaries/Embedded/RegionsHierarchy.cpp @@ -23,7 +23,7 @@ RegionsHierarchy::RegionsHierarchy(IRegionsHierarchyDataSourcePtr data_source_) void RegionsHierarchy::reload() { - Logger * log = &Logger::get("RegionsHierarchy"); + Poco::Logger * log = &Poco::Logger::get("RegionsHierarchy"); if (!data_source->isModified()) return; diff --git a/src/Dictionaries/Embedded/RegionsNames.cpp b/src/Dictionaries/Embedded/RegionsNames.cpp index b31debdbc26..30ba8259b3e 100644 --- a/src/Dictionaries/Embedded/RegionsNames.cpp +++ b/src/Dictionaries/Embedded/RegionsNames.cpp @@ -42,7 +42,7 @@ std::string RegionsNames::dumpSupportedLanguagesNames() void RegionsNames::reload() { - Logger * log = &Logger::get("RegionsNames"); + Poco::Logger * log = &Poco::Logger::get("RegionsNames"); LOG_DEBUG(log, "Reloading regions names"); RegionID max_region_id = 0; diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 0bb8c8543e8..38965e00a84 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -53,7 +53,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( const std::string & config_prefix, Block & sample_block_, const Context & context_) - : log(&Logger::get("ExecutableDictionarySource")) + : log(&Poco::Logger::get("ExecutableDictionarySource")) , dict_struct{dict_struct_} , command{config.getString(config_prefix + ".command")} , update_field{config.getString(config_prefix + ".update_field", "")} @@ -64,7 +64,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( } ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other) - : log(&Logger::get("ExecutableDictionarySource")) + : log(&Poco::Logger::get("ExecutableDictionarySource")) , update_time{other.update_time} , dict_struct{other.dict_struct} , command{other.command} diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 7600253d5b8..95aab78ba2b 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -31,7 +31,7 @@ HTTPDictionarySource::HTTPDictionarySource( Block & sample_block_, const Context & context_, bool check_config) - : log(&Logger::get("HTTPDictionarySource")) + : log(&Poco::Logger::get("HTTPDictionarySource")) , update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , url{config.getString(config_prefix + ".url", "")} @@ -71,7 +71,7 @@ HTTPDictionarySource::HTTPDictionarySource( } HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) - : log(&Logger::get("HTTPDictionarySource")) + : log(&Poco::Logger::get("HTTPDictionarySource")) , update_time{other.update_time} , dict_struct{other.dict_struct} , url{other.url} diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 83d081707bd..ba538201910 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -125,7 +125,7 @@ LibraryDictionarySource::LibraryDictionarySource( Block & sample_block_, const Context & context, bool check_config) - : log(&Logger::get("LibraryDictionarySource")) + : log(&Poco::Logger::get("LibraryDictionarySource")) , dict_struct{dict_struct_} , config_prefix{config_prefix_} , path{config.getString(config_prefix + ".path", "")} @@ -157,7 +157,7 @@ LibraryDictionarySource::LibraryDictionarySource( } LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other) - : log(&Logger::get("LibraryDictionarySource")) + : log(&Poco::Logger::get("LibraryDictionarySource")) , dict_struct{other.dict_struct} , config_prefix{other.config_prefix} , path{other.path} diff --git a/src/Dictionaries/LibraryDictionarySourceExternal.cpp b/src/Dictionaries/LibraryDictionarySourceExternal.cpp index 03447df339c..2e944056283 100644 --- a/src/Dictionaries/LibraryDictionarySourceExternal.cpp +++ b/src/Dictionaries/LibraryDictionarySourceExternal.cpp @@ -10,7 +10,7 @@ void ClickHouseLibrary::log(ClickHouseLibrary::LogLevel level, ClickHouseLibrary { using ClickHouseLibrary::LogLevel; - auto & logger = Logger::get(DICT_LOGGER_NAME); + auto & logger = Poco::Logger::get(DICT_LOGGER_NAME); switch (level) { case LogLevel::TRACE: diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 00f592460a7..505ce7b0c12 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -58,7 +58,7 @@ MySQLDictionarySource::MySQLDictionarySource( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const Block & sample_block_) - : log(&Logger::get("MySQLDictionarySource")) + : log(&Poco::Logger::get("MySQLDictionarySource")) , update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , db{config.getString(config_prefix + ".db", "")} @@ -77,7 +77,7 @@ MySQLDictionarySource::MySQLDictionarySource( /// copy-constructor is provided in order to support cloneability MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other) - : log(&Logger::get("MySQLDictionarySource")) + : log(&Poco::Logger::get("MySQLDictionarySource")) , update_time{other.update_time} , dict_struct{other.dict_struct} , db{other.db} diff --git a/src/Dictionaries/TrieDictionary.h b/src/Dictionaries/TrieDictionary.h index 59f946ebe71..5f8b5df89bf 100644 --- a/src/Dictionaries/TrieDictionary.h +++ b/src/Dictionaries/TrieDictionary.h @@ -253,7 +253,7 @@ private: std::exception_ptr creation_exception; - Logger * logger; + Poco::Logger * logger; }; } diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 03d31fa001f..92af20e646b 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -72,7 +72,7 @@ XDBCDictionarySource::XDBCDictionarySource( const Block & sample_block_, const Context & context_, const BridgeHelperPtr bridge_) - : log(&Logger::get(bridge_->getName() + "DictionarySource")) + : log(&Poco::Logger::get(bridge_->getName() + "DictionarySource")) , update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , db{config_.getString(config_prefix_ + ".db", "")} @@ -96,7 +96,7 @@ XDBCDictionarySource::XDBCDictionarySource( /// copy-constructor is provided in order to support cloneability XDBCDictionarySource::XDBCDictionarySource(const XDBCDictionarySource & other) - : log(&Logger::get(other.bridge_helper->getName() + "DictionarySource")) + : log(&Poco::Logger::get(other.bridge_helper->getName() + "DictionarySource")) , update_time{other.update_time} , dict_struct{other.dict_struct} , db{other.db} diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index d7f1df5ba3b..68f5ee99a7a 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -91,7 +91,7 @@ bool DiskLocal::tryReserve(UInt64 bytes) std::lock_guard lock(DiskLocal::reservation_mutex); if (bytes == 0) { - LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving 0 bytes on disk {}", backQuote(name)); + LOG_DEBUG(&Poco::Logger::get("DiskLocal"), "Reserving 0 bytes on disk {}", backQuote(name)); ++reservation_count; return true; } @@ -100,7 +100,7 @@ bool DiskLocal::tryReserve(UInt64 bytes) UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes); if (unreserved_space >= bytes) { - LOG_DEBUG(&Logger::get("DiskLocal"), "Reserving {} on disk {}, having unreserved {}.", + LOG_DEBUG(&Poco::Logger::get("DiskLocal"), "Reserving {} on disk {}, having unreserved {}.", ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space)); ++reservation_count; reserved_bytes += bytes; @@ -310,7 +310,7 @@ DiskLocalReservation::~DiskLocalReservation() if (disk->reserved_bytes < size) { disk->reserved_bytes = 0; - LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName()); + LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName()); } else { @@ -318,7 +318,7 @@ DiskLocalReservation::~DiskLocalReservation() } if (disk->reservation_count == 0) - LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName()); + LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName()); else --disk->reservation_count; } diff --git a/src/Disks/DiskSelector.cpp b/src/Disks/DiskSelector.cpp index 69549e4520d..0fb728a4f02 100644 --- a/src/Disks/DiskSelector.cpp +++ b/src/Disks/DiskSelector.cpp @@ -99,7 +99,7 @@ DiskSelectorPtr DiskSelector::updateFromConfig( } writeString(" disappeared from configuration, this change will be applied after restart of ClickHouse", warning); - LOG_WARNING(&Logger::get("DiskSelector"), warning.str()); + LOG_WARNING(&Poco::Logger::get("DiskSelector"), warning.str()); } return result; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 2d48db8bc50..71b5991f770 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -515,7 +515,7 @@ std::unique_ptr DiskS3::readFile(const String & path, si { Metadata metadata(metadata_path, path); - LOG_DEBUG(&Logger::get("DiskS3"), "Read from file by path: {}. Existing S3 objects: {}", + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Read from file by path: {}. Existing S3 objects: {}", backQuote(metadata_path + path), metadata.s3_objects.size()); return std::make_unique(client, bucket, metadata, buf_size); @@ -536,7 +536,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, /// Save empty metadata to disk to have ability to get file size while buffer is not finalized. metadata.save(); - LOG_DEBUG(&Logger::get("DiskS3"), "Write to file by path: {} New S3 path: {}", backQuote(metadata_path + path), s3_path); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Write to file by path: {} New S3 path: {}", backQuote(metadata_path + path), s3_path); return std::make_unique(client, bucket, metadata, s3_path, min_upload_part_size, buf_size); } @@ -544,7 +544,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, { Metadata metadata(metadata_path, path); - LOG_DEBUG(&Logger::get("DiskS3"), "Append to file by path: {}. New S3 path: {}. Existing S3 objects: {}.", + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Append to file by path: {}. New S3 path: {}. Existing S3 objects: {}.", backQuote(metadata_path + path), s3_path, metadata.s3_objects.size()); return std::make_unique(client, bucket, metadata, s3_path, min_upload_part_size, buf_size); @@ -553,7 +553,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, void DiskS3::remove(const String & path) { - LOG_DEBUG(&Logger::get("DiskS3"), "Remove file by path: {}", backQuote(metadata_path + path)); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Remove file by path: {}", backQuote(metadata_path + path)); Poco::File file(metadata_path + path); if (file.isFile()) @@ -607,7 +607,7 @@ bool DiskS3::tryReserve(UInt64 bytes) std::lock_guard lock(reservation_mutex); if (bytes == 0) { - LOG_DEBUG(&Logger::get("DiskS3"), "Reserving 0 bytes on s3 disk {}", backQuote(name)); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Reserving 0 bytes on s3 disk {}", backQuote(name)); ++reservation_count; return true; } @@ -616,7 +616,7 @@ bool DiskS3::tryReserve(UInt64 bytes) UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes); if (unreserved_space >= bytes) { - LOG_DEBUG(&Logger::get("DiskS3"), "Reserving {} on disk {}, having unreserved {}.", + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Reserving {} on disk {}, having unreserved {}.", ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space)); ++reservation_count; reserved_bytes += bytes; @@ -672,7 +672,7 @@ DiskS3Reservation::~DiskS3Reservation() if (disk->reserved_bytes < size) { disk->reserved_bytes = 0; - LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName()); + LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'.", disk->getName()); } else { @@ -680,7 +680,7 @@ DiskS3Reservation::~DiskS3Reservation() } if (disk->reservation_count == 0) - LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName()); + LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'.", disk->getName()); else --disk->reservation_count; } diff --git a/src/Disks/S3/ProxyListConfiguration.cpp b/src/Disks/S3/ProxyListConfiguration.cpp index 430b6412408..318b43a22dc 100644 --- a/src/Disks/S3/ProxyListConfiguration.cpp +++ b/src/Disks/S3/ProxyListConfiguration.cpp @@ -20,7 +20,7 @@ Aws::Client::ClientConfigurationPerRequest ProxyListConfiguration::getConfigurat cfg.proxyHost = proxies[index].getHost(); cfg.proxyPort = proxies[index].getPort(); - LOG_DEBUG(&Logger::get("AWSClient"), "Use proxy: {}", proxies[index].toString()); + LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Use proxy: {}", proxies[index].toString()); return cfg; } diff --git a/src/Disks/S3/ProxyResolverConfiguration.cpp b/src/Disks/S3/ProxyResolverConfiguration.cpp index c0cbe4ac5bf..c11697fda0b 100644 --- a/src/Disks/S3/ProxyResolverConfiguration.cpp +++ b/src/Disks/S3/ProxyResolverConfiguration.cpp @@ -21,7 +21,7 @@ ProxyResolverConfiguration::ProxyResolverConfiguration(const Poco::URI & endpoin Aws::Client::ClientConfigurationPerRequest ProxyResolverConfiguration::getConfiguration(const Aws::Http::HttpRequest &) { - LOG_DEBUG(&Logger::get("AWSClient"), "Obtain proxy using resolver: {}", endpoint.toString()); + LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Obtain proxy using resolver: {}", endpoint.toString()); /// 1 second is enough for now. /// TODO: Make timeouts configurable. @@ -49,7 +49,7 @@ Aws::Client::ClientConfigurationPerRequest ProxyResolverConfiguration::getConfig /// Read proxy host as string from response body. Poco::StreamCopier::copyToString(response_body_stream, proxy_host); - LOG_DEBUG(&Logger::get("AWSClient"), "Use proxy: {}://{}:{}", proxy_scheme, proxy_host, proxy_port); + LOG_DEBUG(&Poco::Logger::get("AWSClient"), "Use proxy: {}://{}:{}", proxy_scheme, proxy_host, proxy_port); cfg.proxyScheme = Aws::Http::SchemeMapper::FromString(proxy_scheme.c_str()); cfg.proxyHost = proxy_host; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 53292bb30bc..999a81bd413 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -46,7 +46,7 @@ namespace throw Exception("Only HTTP/HTTPS schemas allowed in proxy resolver config: " + proxy_scheme, ErrorCodes::BAD_ARGUMENTS); auto proxy_port = proxy_resolver_config.getUInt(prefix + ".proxy_port"); - LOG_DEBUG(&Logger::get("DiskS3"), "Configured proxy resolver: {}, Scheme: {}, Port: {}", endpoint.toString(), proxy_scheme, proxy_port); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Configured proxy resolver: {}, Scheme: {}, Port: {}", endpoint.toString(), proxy_scheme, proxy_port); return std::make_shared(endpoint, proxy_scheme, proxy_port); } @@ -70,7 +70,7 @@ namespace proxies.push_back(proxy_uri); - LOG_DEBUG(&Logger::get("DiskS3"), "Configured proxy: {}", proxy_uri.toString()); + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Configured proxy: {}", proxy_uri.toString()); } if (!proxies.empty()) diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index 3f01ed38e1c..f684dce0496 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -245,7 +245,7 @@ StoragePolicySelector::StoragePolicySelector( "StoragePolicy name can contain only alphanumeric and '_' (" + name + ")", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); policies.emplace(name, std::make_shared(name, config, config_prefix + "." + name, disks)); - LOG_INFO(&Logger::get("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name)); + LOG_INFO(&Poco::Logger::get("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name)); } constexpr auto default_storage_policy_name = "default"; diff --git a/src/Disks/VolumeJBOD.cpp b/src/Disks/VolumeJBOD.cpp index eeea763229d..7312f3d2365 100644 --- a/src/Disks/VolumeJBOD.cpp +++ b/src/Disks/VolumeJBOD.cpp @@ -20,7 +20,7 @@ VolumeJBOD::VolumeJBOD( DiskSelectorPtr disk_selector ) : IVolume(name_, config, config_prefix, disk_selector) { - Logger * logger = &Logger::get("StorageConfiguration"); + Poco::Logger * logger = &Poco::Logger::get("StorageConfiguration"); auto has_max_bytes = config.has(config_prefix + ".max_data_part_size_bytes"); auto has_max_ratio = config.has(config_prefix + ".max_data_part_size_ratio"); diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 14c97ee56f4..088ca0c246e 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -173,7 +173,7 @@ namespace auto msg = Poco::AnyCast(session_data); if (!msg.empty()) { - LOG_TRACE((&Logger::get("HTTPCommon")), "Failed communicating with {} with error '{}' will try to reconnect session", host, msg); + LOG_TRACE((&Poco::Logger::get("HTTPCommon")), "Failed communicating with {} with error '{}' will try to reconnect session", host, msg); /// Host can change IP const auto ip = DNSResolver::instance().resolveHost(host).toString(); if (ip != session->getHost()) diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 15d3b4f0beb..829b73d0af6 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -33,7 +33,7 @@ private: Aws::S3::Model::GetObjectResult read_result; std::unique_ptr impl; - Logger * log = &Logger::get("ReadBufferFromS3"); + Poco::Logger * log = &Poco::Logger::get("ReadBufferFromS3"); public: explicit ReadBufferFromS3( diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index 962c663a687..edd0b7f1579 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -127,7 +127,7 @@ namespace detail if (!credentials.getUsername().empty()) credentials.authenticate(request); - LOG_TRACE((&Logger::get("ReadWriteBufferFromHTTP")), "Sending request to {}", uri.toString()); + LOG_TRACE((&Poco::Logger::get("ReadWriteBufferFromHTTP")), "Sending request to {}", uri.toString()); auto sess = session->getSession(); diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 3c2c5835078..b67a1723aca 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -16,16 +16,17 @@ namespace { -const std::pair & convertLogLevel(Aws::Utils::Logging::LogLevel log_level) +const std::pair & convertLogLevel(Aws::Utils::Logging::LogLevel log_level) { - static const std::unordered_map> mapping = { - {Aws::Utils::Logging::LogLevel::Off, {LogsLevel::none, Message::PRIO_FATAL}}, - {Aws::Utils::Logging::LogLevel::Fatal, {LogsLevel::error, Message::PRIO_FATAL}}, - {Aws::Utils::Logging::LogLevel::Error, {LogsLevel::error, Message::PRIO_ERROR}}, - {Aws::Utils::Logging::LogLevel::Warn, {LogsLevel::warning, Message::PRIO_WARNING}}, - {Aws::Utils::Logging::LogLevel::Info, {LogsLevel::information, Message::PRIO_INFORMATION}}, - {Aws::Utils::Logging::LogLevel::Debug, {LogsLevel::debug, Message::PRIO_DEBUG}}, - {Aws::Utils::Logging::LogLevel::Trace, {LogsLevel::trace, Message::PRIO_TRACE}}, + static const std::unordered_map> mapping = + { + {Aws::Utils::Logging::LogLevel::Off, {DB::LogsLevel::none, Poco::Message::PRIO_FATAL}}, + {Aws::Utils::Logging::LogLevel::Fatal, {DB::LogsLevel::error, Poco::Message::PRIO_FATAL}}, + {Aws::Utils::Logging::LogLevel::Error, {DB::LogsLevel::error, Poco::Message::PRIO_ERROR}}, + {Aws::Utils::Logging::LogLevel::Warn, {DB::LogsLevel::warning, Poco::Message::PRIO_WARNING}}, + {Aws::Utils::Logging::LogLevel::Info, {DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION}}, + {Aws::Utils::Logging::LogLevel::Debug, {DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG}}, + {Aws::Utils::Logging::LogLevel::Trace, {DB::LogsLevel::trace, Poco::Message::PRIO_TRACE}}, }; return mapping.at(log_level); } diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index 573ac82a1d3..70cbd7484b4 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -15,7 +15,7 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( request.setHost(uri.getHost()); request.setChunkedTransferEncoding(true); - LOG_TRACE((&Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString()); + LOG_TRACE((&Poco::Logger::get("WriteBufferToHTTP")), "Sending request to {}", uri.toString()); ostr = &session->sendRequest(request); } diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index f928941e482..71c176749af 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -36,7 +36,7 @@ private: String upload_id; std::vector part_tags; - Logger * log = &Logger::get("WriteBufferFromS3"); + Poco::Logger * log = &Poco::Logger::get("WriteBufferFromS3"); public: explicit WriteBufferFromS3( diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 7de1445c960..3b1258052ec 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1565,7 +1565,7 @@ public: ~MergingAndConvertingBlockInputStream() override { - LOG_TRACE(&Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish"); + LOG_TRACE(&Poco::Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish"); /// We need to wait for threads to finish before destructor of 'parallel_merge_data', /// because the threads access 'parallel_merge_data'. diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 1fa0ff282d6..abc1356787e 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1048,7 +1048,7 @@ protected: std::mutex mutex; - Logger * log = &Logger::get("Aggregator"); + Poco::Logger * log = &Poco::Logger::get("Aggregator"); /// Returns true if you can abort the current task. CancellationHook isCancelled; diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 5460b4b3286..5d41b0e87ce 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -180,7 +180,7 @@ void SelectStreamFactory::createForShard( ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); if (shard_info.hasRemoteConnections()) { - LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "There is no table {} on local replica of shard {}, will try remote replicas.", main_table.getNameForLogs(), shard_info.shard_num); + LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), "There is no table {} on local replica of shard {}, will try remote replicas.", main_table.getNameForLogs(), shard_info.shard_num); emplace_remote_stream(); } else @@ -216,7 +216,7 @@ void SelectStreamFactory::createForShard( /// If we reached this point, local replica is stale. ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); - LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "Local replica of shard {} is stale (delay: {}s.)", shard_info.shard_num, local_delay); + LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), "Local replica of shard {} is stale (delay: {}s.)", shard_info.shard_num, local_delay); if (!settings.fallback_to_stale_replicas_for_distributed_queries) { @@ -264,7 +264,7 @@ void SelectStreamFactory::createForShard( catch (const Exception & ex) { if (ex.code() == ErrorCodes::ALL_CONNECTION_TRIES_FAILED) - LOG_WARNING(&Logger::get("ClusterProxy::SelectStreamFactory"), "Connections to remote replicas of local shard {} failed, will use stale local replica", shard_num); + LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), "Connections to remote replicas of local shard {} failed, will use stale local replica", shard_num); else throw; } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9c78fe59da1..5e2f4ecadab 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -287,7 +287,7 @@ void NamedSession::release() */ struct ContextShared { - Logger * log = &Logger::get("Context"); + Poco::Logger * log = &Poco::Logger::get("Context"); /// For access of most of shared objects. Recursive mutex. mutable std::recursive_mutex mutex; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index ee91d903416..dac51b21081 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -220,7 +220,7 @@ static bool isSupportedAlterType(int type) DDLWorker::DDLWorker(const std::string & zk_root_dir, Context & context_, const Poco::Util::AbstractConfiguration * config, const String & prefix) - : context(context_), log(&Logger::get("DDLWorker")) + : context(context_), log(&Poco::Logger::get("DDLWorker")) { queue_dir = zk_root_dir; if (queue_dir.back() == '/') @@ -1073,7 +1073,7 @@ class DDLQueryStatusInputStream : public IBlockInputStream public: DDLQueryStatusInputStream(const String & zk_node_path, const DDLLogEntry & entry, const Context & context_) - : node_path(zk_node_path), context(context_), watch(CLOCK_MONOTONIC_COARSE), log(&Logger::get("DDLQueryStatusInputStream")) + : node_path(zk_node_path), context(context_), watch(CLOCK_MONOTONIC_COARSE), log(&Poco::Logger::get("DDLQueryStatusInputStream")) { sample = Block{ {std::make_shared(), "host"}, @@ -1235,7 +1235,7 @@ private: String node_path; const Context & context; Stopwatch watch; - Logger * log; + Poco::Logger * log; Block sample; diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h index 2f63d9dadee..c30f888b5a1 100644 --- a/src/Interpreters/DDLWorker.h +++ b/src/Interpreters/DDLWorker.h @@ -101,7 +101,7 @@ private: private: Context & context; - Logger * log; + Poco::Logger * log; std::unique_ptr current_context; std::string host_fqdn; /// current host domain name diff --git a/src/Interpreters/EmbeddedDictionaries.cpp b/src/Interpreters/EmbeddedDictionaries.cpp index 9ab3cf2dcbe..58b21fdda52 100644 --- a/src/Interpreters/EmbeddedDictionaries.cpp +++ b/src/Interpreters/EmbeddedDictionaries.cpp @@ -124,7 +124,7 @@ EmbeddedDictionaries::EmbeddedDictionaries( std::unique_ptr geo_dictionaries_loader_, Context & context_, const bool throw_on_error) - : log(&Logger::get("EmbeddedDictionaries")) + : log(&Poco::Logger::get("EmbeddedDictionaries")) , context(context_) , geo_dictionaries_loader(std::move(geo_dictionaries_loader_)) , reload_period(context_.getConfigRef().getInt("builtin_dictionaries_reload_interval", 3600)) diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 4e958a8c12b..ebbac4c5471 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -15,7 +15,7 @@ namespace DB /// Must not acquire Context lock in constructor to avoid possibility of deadlocks. ExternalDictionariesLoader::ExternalDictionariesLoader(Context & context_) - : ExternalLoader("external dictionary", &Logger::get("ExternalDictionariesLoader")) + : ExternalLoader("external dictionary", &Poco::Logger::get("ExternalDictionariesLoader")) , context(context_) { setConfigSettings({"dictionary", "name", "database"}); diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index fc6505e8446..cd46845e2ed 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -100,7 +100,7 @@ namespace class ExternalLoader::LoadablesConfigReader : private boost::noncopyable { public: - LoadablesConfigReader(const String & type_name_, Logger * log_) + LoadablesConfigReader(const String & type_name_, Poco::Logger * log_) : type_name(type_name_), log(log_) { } @@ -366,7 +366,7 @@ private: } const String type_name; - Logger * log; + Poco::Logger * log; std::mutex mutex; ExternalLoaderConfigSettings settings; @@ -389,7 +389,7 @@ public: LoadingDispatcher( const CreateObjectFunction & create_object_function_, const String & type_name_, - Logger * log_) + Poco::Logger * log_) : create_object(create_object_function_) , type_name(type_name_) , log(log_) @@ -1140,7 +1140,7 @@ private: const CreateObjectFunction create_object; const String type_name; - Logger * log; + Poco::Logger * log; mutable std::mutex mutex; std::condition_variable event; @@ -1220,7 +1220,7 @@ private: }; -ExternalLoader::ExternalLoader(const String & type_name_, Logger * log_) +ExternalLoader::ExternalLoader(const String & type_name_, Poco::Logger * log_) : config_files_reader(std::make_unique(type_name_, log_)) , loading_dispatcher(std::make_unique( [this](auto && a, auto && b, auto && c) { return createObject(a, b, c); }, diff --git a/src/Interpreters/ExternalLoader.h b/src/Interpreters/ExternalLoader.h index 9f9fa97b156..542a40e6cb2 100644 --- a/src/Interpreters/ExternalLoader.h +++ b/src/Interpreters/ExternalLoader.h @@ -82,7 +82,7 @@ public: template static constexpr bool is_vector_load_result_type = std::is_same_v || std::is_same_v; - ExternalLoader(const String & type_name_, Logger * log); + ExternalLoader(const String & type_name_, Poco::Logger * log); virtual ~ExternalLoader(); /// Adds a repository which will be used to read configurations from. diff --git a/src/Interpreters/ExternalModelsLoader.cpp b/src/Interpreters/ExternalModelsLoader.cpp index 31356e34855..0300bd44949 100644 --- a/src/Interpreters/ExternalModelsLoader.cpp +++ b/src/Interpreters/ExternalModelsLoader.cpp @@ -11,7 +11,7 @@ namespace ErrorCodes ExternalModelsLoader::ExternalModelsLoader(Context & context_) - : ExternalLoader("external model", &Logger::get("ExternalModelsLoader")) + : ExternalLoader("external model", &Poco::Logger::get("ExternalModelsLoader")) , context(context_) { setConfigSettings({"model", "name", {}}); diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index cc2d62a68fe..5fd2239d104 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -203,7 +203,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s , any_take_last_row(any_take_last_row_) , asof_inequality(table_join->getAsofInequality()) , data(std::make_shared()) - , log(&Logger::get("HashJoin")) + , log(&Poco::Logger::get("HashJoin")) { setSampleBlock(right_sample_block); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 56532e532d1..581c3f3243b 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -201,7 +201,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( , storage(storage_) , input(input_) , input_pipe(std::move(input_pipe_)) - , log(&Logger::get("InterpreterSelectQuery")) + , log(&Poco::Logger::get("InterpreterSelectQuery")) { checkStackSize(); diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index cc3a4e3de34..b8c3920c3e4 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -139,7 +139,7 @@ bool PartLog::addNewParts(Context & current_context, const PartLog::MutableDataP } catch (...) { - tryLogCurrentException(part_log ? part_log->log : &Logger::get("PartLog"), __PRETTY_FUNCTION__); + tryLogCurrentException(part_log ? part_log->log : &Poco::Logger::get("PartLog"), __PRETTY_FUNCTION__); return false; } diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 9120f5959df..d86b5678f6d 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -84,7 +84,7 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as if (!is_unlimited_query && max_size && processes.size() >= max_size) { if (queue_max_wait_ms) - LOG_WARNING(&Logger::get("ProcessList"), "Too many simultaneous queries, will wait {} ms.", queue_max_wait_ms); + LOG_WARNING(&Poco::Logger::get("ProcessList"), "Too many simultaneous queries, will wait {} ms.", queue_max_wait_ms); if (!queue_max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(queue_max_wait_ms), [&]{ return processes.size() < max_size; })) throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES); } @@ -231,7 +231,7 @@ ProcessListEntry::~ProcessListEntry() auto user_process_list_it = parent.user_to_queries.find(user); if (user_process_list_it == parent.user_to_queries.end()) { - LOG_ERROR(&Logger::get("ProcessList"), "Logical error: cannot find user in ProcessList"); + LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find user in ProcessList"); std::terminate(); } @@ -250,7 +250,7 @@ ProcessListEntry::~ProcessListEntry() if (!found) { - LOG_ERROR(&Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser"); + LOG_ERROR(&Poco::Logger::get("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser"); std::terminate(); } parent.have_space.notify_all(); diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 848abc9aa8a..a4c8fd59245 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -31,7 +31,7 @@ public: /// store all set elements in explicit form. /// This is needed for subsequent use for index. Set(const SizeLimits & limits_, bool fill_set_elements_, bool transform_null_in_) - : log(&Logger::get("Set")), + : log(&Poco::Logger::get("Set")), limits(limits_), fill_set_elements(fill_set_elements_), transform_null_in(transform_null_in_) { } @@ -105,7 +105,7 @@ private: /// Types for set_elements. DataTypes set_elements_types; - Logger * log; + Poco::Logger * log; /// Limitations on the maximum size of the set SizeLimits limits; diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index e7e65c4f19e..6ee9d3f38df 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -41,7 +41,7 @@ std::shared_ptr createSystemLog( if (database != default_database_name) { /// System tables must be loaded before other tables, but loading order is undefined for all databases except `system` - LOG_ERROR(&Logger::get("SystemLog"), "Custom database name for a system table specified in config. Table `{}` will be created in `system` database instead of `{}`", table, database); + LOG_ERROR(&Poco::Logger::get("SystemLog"), "Custom database name for a system table specified in config. Table `{}` will be created in `system` database instead of `{}`", table, database); database = default_database_name; } diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 649cfa28e6e..dd2f815ce92 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -152,7 +152,7 @@ public: ASTPtr getCreateTableQuery() override; protected: - Logger * log; + Poco::Logger * log; private: /* Saving thread data */ @@ -207,7 +207,7 @@ SystemLog::SystemLog(Context & context_, , flush_interval_milliseconds(flush_interval_milliseconds_) { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); - log = &Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); + log = &Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index e825c1949cf..e1e2108c0fc 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -105,7 +105,7 @@ static void logQuery(const String & query, const Context & context, bool interna { if (internal) { - LOG_DEBUG(&Logger::get("executeQuery"), "(internal) {}", joinLines(query)); + LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(internal) {}", joinLines(query)); } else { @@ -113,7 +113,7 @@ static void logQuery(const String & query, const Context & context, bool interna const auto & initial_query_id = context.getClientInfo().initial_query_id; const auto & current_user = context.getClientInfo().current_user; - LOG_DEBUG(&Logger::get("executeQuery"), "(from {}{}{}) {}", + LOG_DEBUG(&Poco::Logger::get("executeQuery"), "(from {}{}{}) {}", context.getClientInfo().current_address.toString(), (current_user != "default" ? ", user: " + context.getClientInfo().current_user : ""), (!initial_query_id.empty() && current_query_id != initial_query_id ? ", initial_query_id: " + initial_query_id : std::string()), @@ -145,10 +145,10 @@ static void setExceptionStackTrace(QueryLogElement & elem) static void logException(Context & context, QueryLogElement & elem) { if (elem.stack_trace.empty()) - LOG_ERROR(&Logger::get("executeQuery"), "{} (from {}) (in query: {})", + LOG_ERROR(&Poco::Logger::get("executeQuery"), "{} (from {}) (in query: {})", elem.exception, context.getClientInfo().current_address.toString(), joinLines(elem.query)); else - LOG_ERROR(&Logger::get("executeQuery"), "{} (from {}) (in query: {})" + LOG_ERROR(&Poco::Logger::get("executeQuery"), "{} (from {}) (in query: {})" ", Stack trace (when copying this message, always include the lines below):\n\n{}", elem.exception, context.getClientInfo().current_address.toString(), joinLines(elem.query), elem.stack_trace); } @@ -505,7 +505,7 @@ static std::tuple executeQueryImpl( if (elem.read_rows != 0) { - LOG_INFO(&Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", + LOG_INFO(&Poco::Logger::get("executeQuery"), "Read {} rows, {} in {} sec., {} rows/sec., {}/sec.", elem.read_rows, ReadableSize(elem.read_bytes), elapsed_seconds, static_cast(elem.read_rows / elapsed_seconds), ReadableSize(elem.read_bytes / elapsed_seconds)); @@ -585,7 +585,7 @@ static std::tuple executeQueryImpl( std::stringstream log_str; log_str << "Query pipeline:\n"; res.in->dumpTree(log_str); - LOG_DEBUG(&Logger::get("executeQuery"), log_str.str()); + LOG_DEBUG(&Poco::Logger::get("executeQuery"), log_str.str()); } } } diff --git a/src/Interpreters/tests/select_query.cpp b/src/Interpreters/tests/select_query.cpp index 8ce65ede05f..fb364d28086 100644 --- a/src/Interpreters/tests/select_query.cpp +++ b/src/Interpreters/tests/select_query.cpp @@ -24,8 +24,8 @@ int main(int, char **) try { Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); /// Pre-initialize the `DateLUT` so that the first initialization does not affect the measured execution speed. DateLUT::instance(); diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index e2676eb64c2..12d4db1f4a8 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -149,7 +149,7 @@ Chunk IRowInputFormat::generate() { if (num_errors && (params.allow_errors_num > 0 || params.allow_errors_ratio > 0)) { - Logger * log = &Logger::get("IRowInputFormat"); + Poco::Logger * log = &Poco::Logger::get("IRowInputFormat"); LOG_TRACE(log, "Skipped {} rows with errors while reading the input stream", num_errors); } diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 78a291de262..8017667909b 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -612,7 +612,7 @@ private: try { Poco::URI url(base_url, "/schemas/ids/" + std::to_string(id)); - LOG_TRACE((&Logger::get("AvroConfluentRowInputFormat")), "Fetching schema id = {}", id); + LOG_TRACE((&Poco::Logger::get("AvroConfluentRowInputFormat")), "Fetching schema id = {}", id); /// One second for connect/send/receive. Just in case. ConnectionTimeouts timeouts({1, 0}, {1, 0}, {1, 0}); @@ -629,7 +629,7 @@ private: Poco::JSON::Parser parser; auto json_body = parser.parse(*response_body).extract(); auto schema = json_body->getValue("schema"); - LOG_TRACE((&Logger::get("AvroConfluentRowInputFormat")), "Succesfully fetched schema id = {}\n{}", id, schema); + LOG_TRACE((&Poco::Logger::get("AvroConfluentRowInputFormat")), "Succesfully fetched schema id = {}\n{}", id, schema); return avro::compileJsonSchemaFromString(schema); } catch (const Exception &) diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp index 05276f87540..49a3d018098 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.cpp @@ -27,7 +27,7 @@ CollapsingSortedAlgorithm::CollapsingSortedAlgorithm( size_t max_block_size, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes, - Logger * log_) + Poco::Logger * log_) : IMergingAlgorithmWithSharedChunks(num_inputs, std::move(description_), out_row_sources_buf_, max_row_refs) , merged_data(header.cloneEmptyColumns(), use_average_block_sizes, max_block_size) , sign_column_number(header.getPositionByName(sign_column)) diff --git a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h index 3cbe95d96e1..d95fac2f02b 100644 --- a/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h @@ -35,7 +35,7 @@ public: size_t max_block_size, WriteBuffer * out_row_sources_buf_, bool use_average_block_sizes, - Logger * log_); + Poco::Logger * log_); Status merge() override; @@ -62,7 +62,7 @@ private: PODArray current_row_sources; /// Sources of rows with the current primary key size_t count_incorrect_data = 0; /// To prevent too many error messages from writing to the log. - Logger * log; + Poco::Logger * log; void reportIncorrectData(); void insertRow(RowRef & row); diff --git a/src/Processors/Merges/Algorithms/RowRef.h b/src/Processors/Merges/Algorithms/RowRef.h index efd04717e94..1b4da9781f8 100644 --- a/src/Processors/Merges/Algorithms/RowRef.h +++ b/src/Processors/Merges/Algorithms/RowRef.h @@ -83,7 +83,7 @@ public: { if (free_chunks.size() != chunks.size()) { - LOG_ERROR(&Logger::get("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); + LOG_ERROR(&Poco::Logger::get("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); return; } @@ -100,7 +100,7 @@ private: /// This may happen if allocator was removed before chunks. /// Log message and exit, because we don't want to throw exception in destructor. - LOG_ERROR(&Logger::get("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); + LOG_ERROR(&Poco::Logger::get("SharedChunkAllocator"), "SharedChunkAllocator was destroyed before RowRef was released. StackTrace: {}", StackTrace().toString()); return; } diff --git a/src/Processors/Merges/CollapsingSortedTransform.h b/src/Processors/Merges/CollapsingSortedTransform.h index cdf7c4a1607..4e65504a101 100644 --- a/src/Processors/Merges/CollapsingSortedTransform.h +++ b/src/Processors/Merges/CollapsingSortedTransform.h @@ -29,7 +29,7 @@ public: max_block_size, out_row_sources_buf_, use_average_block_sizes, - &Logger::get("CollapsingSortedTransform")) + &Poco::Logger::get("CollapsingSortedTransform")) { } diff --git a/src/Processors/Merges/MergingSortedTransform.cpp b/src/Processors/Merges/MergingSortedTransform.cpp index 9f797b32123..e680304ccee 100644 --- a/src/Processors/Merges/MergingSortedTransform.cpp +++ b/src/Processors/Merges/MergingSortedTransform.cpp @@ -43,7 +43,7 @@ void MergingSortedTransform::onFinish() const auto & merged_data = algorithm.getMergedData(); - auto * log = &Logger::get("MergingSortedTransform"); + auto * log = &Poco::Logger::get("MergingSortedTransform"); double seconds = total_stopwatch.elapsedSeconds(); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index ebfb14a387a..c2693579c67 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -88,7 +88,7 @@ private: Processors processors; AggregatingTransformParamsPtr params; - Logger * log = &Logger::get("AggregatingTransform"); + Poco::Logger * log = &Poco::Logger::get("AggregatingTransform"); ColumnRawPtrs key_columns; Aggregator::AggregateColumns aggregate_columns; diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index aeb7a43b61b..f6df6042959 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -54,7 +54,7 @@ private: size_t bytes_to_transfer = 0; using Logger = Poco::Logger; - Logger * log = &Logger::get("CreatingSetsBlockInputStream"); + Poco::Logger * log = &Poco::Logger::get("CreatingSetsBlockInputStream"); bool is_initialized = false; diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index 279faaec8ac..428fbb6d528 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -31,7 +31,7 @@ class MergeSorter; class BufferingToFileTransform : public IAccumulatingTransform { public: - BufferingToFileTransform(const Block & header, Logger * log_, std::string path_) + BufferingToFileTransform(const Block & header, Poco::Logger * log_, std::string path_) : IAccumulatingTransform(header, header), log(log_) , path(std::move(path_)), file_buf_out(path), compressed_buf_out(file_buf_out) , out_stream(std::make_shared(compressed_buf_out, 0, header)) @@ -80,7 +80,7 @@ public: } private: - Logger * log; + Poco::Logger * log; std::string path; WriteBufferFromFile file_buf_out; CompressedWriteBuffer compressed_buf_out; diff --git a/src/Processors/Transforms/MergeSortingTransform.h b/src/Processors/Transforms/MergeSortingTransform.h index 043cb3f36c1..22812e08b40 100644 --- a/src/Processors/Transforms/MergeSortingTransform.h +++ b/src/Processors/Transforms/MergeSortingTransform.h @@ -41,7 +41,7 @@ private: size_t sum_rows_in_blocks = 0; size_t sum_bytes_in_blocks = 0; - Logger * log = &Logger::get("MergeSortingTransform"); + Poco::Logger * log = &Poco::Logger::get("MergeSortingTransform"); /// If remerge doesn't save memory at least several times, mark it as useless and don't do it anymore. bool remerge_is_useful = true; diff --git a/src/Processors/Transforms/MergingAggregatedTransform.h b/src/Processors/Transforms/MergingAggregatedTransform.h index cb1ce01976c..73e0d8cd013 100644 --- a/src/Processors/Transforms/MergingAggregatedTransform.h +++ b/src/Processors/Transforms/MergingAggregatedTransform.h @@ -21,7 +21,7 @@ protected: private: AggregatingTransformParamsPtr params; - Logger * log = &Logger::get("MergingAggregatedTransform"); + Poco::Logger * log = &Poco::Logger::get("MergingAggregatedTransform"); size_t max_threads; AggregatedDataVariants data_variants; diff --git a/src/Processors/tests/processors_test_aggregation.cpp b/src/Processors/tests/processors_test_aggregation.cpp index e3316432ba8..9b8bee67d52 100644 --- a/src/Processors/tests/processors_test_aggregation.cpp +++ b/src/Processors/tests/processors_test_aggregation.cpp @@ -184,8 +184,8 @@ try auto thread_group = CurrentThread::getGroup(); Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); registerAggregateFunctions(); auto & factory = AggregateFunctionFactory::instance(); diff --git a/src/Processors/tests/processors_test_merge_sorting_transform.cpp b/src/Processors/tests/processors_test_merge_sorting_transform.cpp index 470bf79a174..5e6720f0167 100644 --- a/src/Processors/tests/processors_test_merge_sorting_transform.cpp +++ b/src/Processors/tests/processors_test_merge_sorting_transform.cpp @@ -125,8 +125,8 @@ int main(int, char **) try { Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); auto disk = std::make_shared("tmp", ".", 0); auto tmp_volume = std::make_shared("tmp", std::vector{disk}, 0); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 82068496159..84d23f10a55 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -225,7 +225,7 @@ void HTTPHandler::pushDelayedResults(Output & used_output) HTTPHandler::HTTPHandler(IServer & server_, const std::string & name) : server(server_) - , log(&Logger::get(name)) + , log(&Poco::Logger::get(name)) { server_display_name = server.config().getString("display_name", getFQDNOrHostName()); } diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp index e916070be22..2f00aa0aa72 100644 --- a/src/Server/HTTPHandlerFactory.cpp +++ b/src/Server/HTTPHandlerFactory.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes } HTTPRequestHandlerFactoryMain::HTTPRequestHandlerFactoryMain(const std::string & name_) - : log(&Logger::get(name_)), name(name_) + : log(&Poco::Logger::get(name_)), name(name_) { } diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h index 55206018d41..273e337813e 100644 --- a/src/Server/HTTPHandlerFactory.h +++ b/src/Server/HTTPHandlerFactory.h @@ -18,7 +18,7 @@ class HTTPRequestHandlerFactoryMain : public Poco::Net::HTTPRequestHandlerFactor private: using TThis = HTTPRequestHandlerFactoryMain; - Logger * log; + Poco::Logger * log; std::string name; std::vector child_factories; diff --git a/src/Server/MySQLHandlerFactory.cpp b/src/Server/MySQLHandlerFactory.cpp index 5d78ed81068..18e05f88402 100644 --- a/src/Server/MySQLHandlerFactory.cpp +++ b/src/Server/MySQLHandlerFactory.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes MySQLHandlerFactory::MySQLHandlerFactory(IServer & server_) : server(server_) - , log(&Logger::get("MySQLHandlerFactory")) + , log(&Poco::Logger::get("MySQLHandlerFactory")) { #if USE_SSL try diff --git a/src/Server/ReplicasStatusHandler.cpp b/src/Server/ReplicasStatusHandler.cpp index 986af59d3a4..55746409feb 100644 --- a/src/Server/ReplicasStatusHandler.cpp +++ b/src/Server/ReplicasStatusHandler.cpp @@ -96,7 +96,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request } catch (...) { - LOG_ERROR((&Logger::get("ReplicasStatusHandler")), "Cannot send exception to client"); + LOG_ERROR((&Poco::Logger::get("ReplicasStatusHandler")), "Cannot send exception to client"); } } } diff --git a/src/Server/TCPHandlerFactory.h b/src/Server/TCPHandlerFactory.h index a5532a8dc02..5ecd427bf8b 100644 --- a/src/Server/TCPHandlerFactory.h +++ b/src/Server/TCPHandlerFactory.h @@ -27,7 +27,7 @@ private: public: explicit TCPHandlerFactory(IServer & server_, bool secure_ = false) : server(server_) - , log(&Logger::get(std::string("TCP") + (secure_ ? "S" : "") + "HandlerFactory")) + , log(&Poco::Logger::get(std::string("TCP") + (secure_ ? "S" : "") + "HandlerFactory")) { } diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 4dd62db0965..0e84f68e5fe 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -88,7 +88,7 @@ StorageDistributedDirectoryMonitor::StorageDistributedDirectoryMonitor( , default_sleep_time{storage.global_context->getSettingsRef().distributed_directory_monitor_sleep_time_ms.totalMilliseconds()} , sleep_time{default_sleep_time} , max_sleep_time{storage.global_context->getSettingsRef().distributed_directory_monitor_max_sleep_time_ms.totalMilliseconds()} - , log{&Logger::get(getLoggerName())} + , log{&Poco::Logger::get(getLoggerName())} , monitor_blocker(monitor_blocker_) , bg_pool(bg_pool_) { @@ -301,7 +301,7 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa } void StorageDistributedDirectoryMonitor::readHeader( - ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Logger * log) + ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Poco::Logger * log) { UInt64 query_size; readVarUInt(query_size, in); @@ -542,7 +542,7 @@ public: : in(file_name) , decompressing_in(in) , block_in(decompressing_in, ClickHouseRevision::get()) - , log{&Logger::get("DirectoryMonitorBlockInputStream")} + , log{&Poco::Logger::get("DirectoryMonitorBlockInputStream")} { Settings insert_settings; String insert_query; @@ -576,7 +576,7 @@ private: Block first_block; Block header; - Logger * log; + Poco::Logger * log; }; BlockInputStreamPtr StorageDistributedDirectoryMonitor::createStreamFromFile(const String & file_name) diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index e2a913ee1ef..418cd430243 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -68,14 +68,14 @@ private: std::chrono::time_point last_decrease_time {std::chrono::system_clock::now()}; std::atomic quit {false}; std::mutex mutex; - Logger * log; + Poco::Logger * log; ActionBlocker & monitor_blocker; BackgroundSchedulePool & bg_pool; BackgroundSchedulePoolTaskHolder task_handle; /// Read insert query and insert settings for backward compatible. - static void readHeader(ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Logger * log); + static void readHeader(ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Poco::Logger * log); friend class DirectoryMonitorBlockInputStream; }; diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index a171e289695..5516e85b143 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -87,7 +87,7 @@ DistributedBlockOutputStream::DistributedBlockOutputStream( bool insert_sync_, UInt64 insert_timeout_) : context(context_), storage(storage_), query_ast(query_ast_), query_string(queryToString(query_ast_)), cluster(cluster_), insert_sync(insert_sync_), - insert_timeout(insert_timeout_), log(&Logger::get("DistributedBlockOutputStream")) + insert_timeout(insert_timeout_), log(&Poco::Logger::get("DistributedBlockOutputStream")) { } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 930d6ff55b5..fd0d3a7c85b 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -137,7 +137,7 @@ StorageKafka::StorageKafka( , schema_name(global_context.getMacros()->expand(schema_name_)) , num_consumers(num_consumers_) , max_block_size(max_block_size_) - , log(&Logger::get("StorageKafka (" + table_id_.table_name + ")")) + , log(&Poco::Logger::get("StorageKafka (" + table_id_.table_name + ")")) , semaphore(0, num_consumers_) , skip_broken(skip_broken_) , intermediate_commit(intermediate_commit_) diff --git a/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/src/Storages/MergeTree/BackgroundProcessingPool.cpp index ff195112929..8f6d7c19549 100644 --- a/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -51,7 +51,7 @@ BackgroundProcessingPool::BackgroundProcessingPool(int size_, , thread_name(thread_name_) , settings(pool_settings) { - logger = &Logger::get(log_name); + logger = &Poco::Logger::get(log_name); LOG_INFO(logger, "Create {} with {} threads", log_name, size); threads.resize(size); diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index c0e8c0d2331..c1aff6bdba5 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -21,7 +21,7 @@ class Service final : public InterserverIOEndpoint { public: Service(MergeTreeData & data_) - : data(data_), log(&Logger::get(data.getLogName() + " (Replicated PartsService)")) {} + : data(data_), log(&Poco::Logger::get(data.getLogName() + " (Replicated PartsService)")) {} Service(const Service &) = delete; Service & operator=(const Service &) = delete; @@ -36,7 +36,7 @@ private: /// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish, /// so Service will never access dangling reference to storage MergeTreeData & data; - Logger * log; + Poco::Logger * log; }; /** Client for getting the parts from the table *MergeTree. @@ -44,7 +44,7 @@ private: class Fetcher final { public: - Fetcher(MergeTreeData & data_) : data(data_), log(&Logger::get("Fetcher")) {} + Fetcher(MergeTreeData & data_) : data(data_), log(&Poco::Logger::get("Fetcher")) {} Fetcher(const Fetcher &) = delete; Fetcher & operator=(const Fetcher &) = delete; @@ -75,7 +75,7 @@ private: PooledReadWriteBufferFromHTTP & in); MergeTreeData & data; - Logger * log; + Poco::Logger * log; }; } diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 758401b7f4f..762dbc7d5b6 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -107,7 +107,7 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions( int rc = zookeeper.tryMulti(lock_ops, lock_responses); if (rc == Coordination::ZBADVERSION) { - LOG_TRACE(&Logger::get("EphemeralLocksInAllPartitions"), "Someone has inserted a block in a new partition while we were creating locks. Retry."); + LOG_TRACE(&Poco::Logger::get("EphemeralLocksInAllPartitions"), "Someone has inserted a block in a new partition while we were creating locks. Retry."); continue; } else if (rc != Coordination::ZOK) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 215d5f493e5..76d9c484d76 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -136,7 +136,7 @@ MergeTreeData::MergeTreeData( , relative_data_path(relative_data_path_) , broken_part_callback(broken_part_callback_) , log_name(table_id_.getNameForLogs()) - , log(&Logger::get(log_name)) + , log(&Poco::Logger::get(log_name)) , storage_settings(std::move(storage_settings_)) , data_parts_by_info(data_parts_indexes.get()) , data_parts_by_state_and_info(data_parts_indexes.get()) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index bf9bfea88b3..3fb124b26a0 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -697,7 +697,7 @@ protected: BrokenPartCallback broken_part_callback; String log_name; - Logger * log; + Poco::Logger * log; /// Storage settings. /// Use get and set to receive readonly versions. diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index c83caae3aff..eb8112d1008 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -153,7 +153,7 @@ void FutureMergedMutatedPart::updatePath(const MergeTreeData & storage, const Re } MergeTreeDataMergerMutator::MergeTreeDataMergerMutator(MergeTreeData & data_, size_t background_pool_size_) - : data(data_), background_pool_size(background_pool_size_), log(&Logger::get(data.getLogName() + " (MergerMutator)")) + : data(data_), background_pool_size(background_pool_size_), log(&Poco::Logger::get(data.getLogName() + " (MergerMutator)")) { } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 84a3fcf1be2..d26e84eb18a 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -230,7 +230,7 @@ private: MergeTreeData & data; const size_t background_pool_size; - Logger * log; + Poco::Logger * log; /// When the last time you wrote to the log that the disk space was running out (not to write about this too often). time_t disk_space_warning_time = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 24e3e3ac69d..e4321b82166 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -84,7 +84,7 @@ namespace ErrorCodes MergeTreeDataSelectExecutor::MergeTreeDataSelectExecutor(const MergeTreeData & data_) - : data(data_), log(&Logger::get(data.getLogName() + " (SelectExecutor)")) + : data(data_), log(&Poco::Logger::get(data.getLogName() + " (SelectExecutor)")) { } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 92fa98fd914..942e111635b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -44,7 +44,7 @@ public: private: const MergeTreeData & data; - Logger * log; + Poco::Logger * log; Pipes spreadMarkRangesAmongStreams( RangesInDataParts && parts, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index e199aa3b43a..ffaa227641e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -34,7 +34,7 @@ using BlocksWithPartition = std::vector; class MergeTreeDataWriter { public: - MergeTreeDataWriter(MergeTreeData & data_) : data(data_), log(&Logger::get(data.getLogName() + " (Writer)")) {} + MergeTreeDataWriter(MergeTreeData & data_) : data(data_), log(&Poco::Logger::get(data.getLogName() + " (Writer)")) {} /** Split the block to blocks, each of them must be written as separate part. * (split rows by partition) @@ -50,7 +50,7 @@ public: private: MergeTreeData & data; - Logger * log; + Poco::Logger * log; }; } diff --git a/src/Storages/MergeTree/MergeTreePartsMover.h b/src/Storages/MergeTree/MergeTreePartsMover.h index 95a20dc1f77..8ddf5e165af 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.h +++ b/src/Storages/MergeTree/MergeTreePartsMover.h @@ -66,7 +66,7 @@ public: private: MergeTreeData * data; - Logger * log; + Poco::Logger * log; }; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index f639a6a4905..c43074f1962 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -133,7 +133,7 @@ private: mutable std::mutex mutex; - Logger * log = &Logger::get("MergeTreeReadPool"); + Poco::Logger * log = &Poco::Logger::get("MergeTreeReadPool"); }; using MergeTreeReadPoolPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h index 211bf9701cf..ea603bd468f 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h @@ -69,7 +69,7 @@ private: Chunks chunks; - Logger * log = &Logger::get("MergeTreeReverseSelectProcessor"); + Poco::Logger * log = &Poco::Logger::get("MergeTreeReverseSelectProcessor"); }; } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 4c64bfb6a18..d2438e20192 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -67,7 +67,7 @@ private: String path; bool is_first_task = true; - Logger * log = &Logger::get("MergeTreeSelectProcessor"); + Poco::Logger * log = &Poco::Logger::get("MergeTreeSelectProcessor"); }; } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index dac559913aa..6155fef200a 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -45,7 +45,7 @@ private: /// Should read using direct IO bool read_with_direct_io; - Logger * log = &Logger::get("MergeTreeSequentialSource"); + Poco::Logger * log = &Poco::Logger::get("MergeTreeSequentialSource"); std::shared_ptr mark_cache; using MergeTreeReaderPtr = std::unique_ptr; diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index a8da0e8615c..61f99ac6d88 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -32,7 +32,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( const Context & context, const MergeTreeData & data, const Names & queried_columns_, - Logger * log_) + Poco::Logger * log_) : table_columns{ext::map(data.getColumns().getAllPhysical(), [] (const NameAndTypePair & col) { return col.name; })}, queried_columns{queried_columns_}, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index d5e44166536..d8c1103809f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -33,7 +33,7 @@ namespace ErrorCodes ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream( StorageReplicatedMergeTree & storage_, size_t quorum_, size_t quorum_timeout_ms_, size_t max_parts_per_block_, bool deduplicate_) : storage(storage_), quorum(quorum_), quorum_timeout_ms(quorum_timeout_ms_), max_parts_per_block(max_parts_per_block_), deduplicate(deduplicate_), - log(&Logger::get(storage.getLogName() + " (Replicated OutputStream)")) + log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) { /// The quorum value `1` has the same meaning as if it is disabled. if (quorum == 1) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index 0f6fc1e7cee..b8650c25c7d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -63,7 +63,7 @@ private: bool last_block_is_duplicate = false; using Logger = Poco::Logger; - Logger * log; + Poco::Logger * log; }; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 5fc2e2e3417..de91a5d5940 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes ReplicatedMergeTreeCleanupThread::ReplicatedMergeTreeCleanupThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeCleanupThread)") - , log(&Logger::get(log_name)) + , log(&Poco::Logger::get(log_name)) { task = storage.global_context.getSchedulePool().createTask(log_name, [this]{ run(); }); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h index 306ee29a5de..a787f99d907 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h @@ -34,7 +34,7 @@ public: private: StorageReplicatedMergeTree & storage; String log_name; - Logger * log; + Poco::Logger * log; BackgroundSchedulePool::TaskHolder task; pcg64 rng; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp index eeb6b9bddd7..8f99f315620 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp @@ -26,7 +26,7 @@ static const auto PART_CHECK_ERROR_SLEEP_MS = 5 * 1000; ReplicatedMergeTreePartCheckThread::ReplicatedMergeTreePartCheckThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreePartCheckThread)") - , log(&Logger::get(log_name)) + , log(&Poco::Logger::get(log_name)) { task = storage.global_context.getSchedulePool().createTask(log_name, [this] { run(); }); task->schedule(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h index 198c9714f64..e86356e1346 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h @@ -77,7 +77,7 @@ private: StorageReplicatedMergeTree & storage; String log_name; - Logger * log; + Poco::Logger * log; using StringSet = std::set; using PartToCheck = std::pair; /// The name of the part and the minimum time to check (or zero, if not important). diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 9c8d1e9ac5c..4ea7ddda738 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -116,7 +116,7 @@ void ReplicatedMergeTreeQueue::initialize( zookeeper_path = zookeeper_path_; replica_path = replica_path_; logger_name = logger_name_; - log = &Logger::get(logger_name); + log = &Poco::Logger::get(logger_name); addVirtualParts(parts); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 8e58c8b7af2..4cbb86adb7b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -56,7 +56,7 @@ private: String zookeeper_path; String replica_path; String logger_name; - Logger * log = nullptr; + Poco::Logger * log = nullptr; /// Protects the queue, future_parts and other queue state variables. mutable std::mutex state_mutex; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 5b4f4f2a5be..93d652f2be0 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -42,7 +42,7 @@ static String generateActiveNodeIdentifier() ReplicatedMergeTreeRestartingThread::ReplicatedMergeTreeRestartingThread(StorageReplicatedMergeTree & storage_) : storage(storage_) , log_name(storage.getStorageID().getFullTableName() + " (ReplicatedMergeTreeRestartingThread)") - , log(&Logger::get(log_name)) + , log(&Poco::Logger::get(log_name)) , active_node_identifier(generateActiveNodeIdentifier()) { const auto storage_settings = storage.getSettings(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 88c6fe755d1..8641af07476 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -33,7 +33,7 @@ public: private: StorageReplicatedMergeTree & storage; String log_name; - Logger * log; + Poco::Logger * log; std::atomic need_stop {false}; /// The random data we wrote into `/replicas/me/is_active`. diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 1072f0f3537..720c7a2d670 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -74,7 +74,7 @@ StorageBuffer::StorageBuffer( , max_thresholds(max_thresholds_) , destination_id(destination_id_) , allow_materialized(allow_materialized_) - , log(&Logger::get("StorageBuffer (" + table_id_.getFullTableName() + ")")) + , log(&Poco::Logger::get("StorageBuffer (" + table_id_.getFullTableName() + ")")) , bg_pool(global_context.getBufferFlushSchedulePool()) { setColumns(columns_); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 62755c99642..574d93df566 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -278,7 +278,7 @@ StorageDistributed::StorageDistributed( , remote_database(remote_database_) , remote_table(remote_table_) , global_context(std::make_unique(context_)) - , log(&Logger::get("StorageDistributed (" + id_.table_name + ")")) + , log(&Poco::Logger::get("StorageDistributed (" + id_.table_name + ")")) , cluster_name(global_context->getMacros()->expand(cluster_name_)) , has_sharding_key(sharding_key_) , storage_policy(storage_policy_) diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h index 5b91c800a1f..a7e3a073af4 100644 --- a/src/Storages/StorageDistributed.h +++ b/src/Storages/StorageDistributed.h @@ -128,7 +128,7 @@ public: ASTPtr remote_table_function_ptr; std::unique_ptr global_context; - Logger * log; + Poco::Logger * log; /// Used to implement TableFunctionRemote. std::shared_ptr owned_cluster; diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 33d8385cf85..fa5034d946c 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -85,7 +85,7 @@ private: mutable std::shared_mutex rwlock; - Logger * log = &Logger::get("StorageFile"); + Poco::Logger * log = &Poco::Logger::get("StorageFile"); }; } diff --git a/src/Storages/StorageHDFS.h b/src/Storages/StorageHDFS.h index 48defd72911..5b250247b84 100644 --- a/src/Storages/StorageHDFS.h +++ b/src/Storages/StorageHDFS.h @@ -45,7 +45,7 @@ private: Context & context; String compression_method; - Logger * log = &Logger::get("StorageHDFS"); + Poco::Logger * log = &Poco::Logger::get("StorageHDFS"); }; } diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index a1ba39ef158..86bfed5ac84 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -191,7 +191,7 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) backup_stream.readSuffix(); /// TODO Add speed, compressed bytes, data volume in memory, compression ratio ... Generalize all statistics logging in project. - LOG_INFO(&Logger::get("StorageSetOrJoinBase"), "Loaded from backup file {}. {} rows, {}. State has {} unique rows.", + LOG_INFO(&Poco::Logger::get("StorageSetOrJoinBase"), "Loaded from backup file {}. {} rows, {}. State has {} unique rows.", file_path, backup_stream.getProfileInfo().rows, ReadableSize(backup_stream.getProfileInfo().bytes), getSize()); } diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 5c3e93fc7f8..b61d52657dd 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -221,7 +221,7 @@ StorageStripeLog::StorageStripeLog( , table_path(relative_path_) , max_compress_block_size(max_compress_block_size_) , file_checker(disk, table_path + "sizes.json") - , log(&Logger::get("StorageStripeLog")) + , log(&Poco::Logger::get("StorageStripeLog")) { setColumns(columns_); setConstraints(constraints_); diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index c85f576bd43..ed8e5da081e 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -68,7 +68,7 @@ private: FileChecker file_checker; mutable std::shared_mutex rwlock; - Logger * log; + Poco::Logger * log; }; } diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 5bdfa1fc374..2a62068516e 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -334,7 +334,7 @@ StorageTinyLog::StorageTinyLog( , table_path(relative_path_) , max_compress_block_size(max_compress_block_size_) , file_checker(disk, table_path + "sizes.json") - , log(&Logger::get("StorageTinyLog")) + , log(&Poco::Logger::get("StorageTinyLog")) { setColumns(columns_); setConstraints(constraints_); diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index a8be3be2435..102ec76fda3 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -71,7 +71,7 @@ private: FileChecker file_checker; mutable std::shared_mutex rwlock; - Logger * log; + Poco::Logger * log; void addFiles(const String & column_name, const IDataType & type); }; diff --git a/utils/zookeeper-cli/zookeeper-cli.cpp b/utils/zookeeper-cli/zookeeper-cli.cpp index 0a503e77250..6fd7b39ab68 100644 --- a/utils/zookeeper-cli/zookeeper-cli.cpp +++ b/utils/zookeeper-cli/zookeeper-cli.cpp @@ -66,8 +66,8 @@ int main(int argc, char ** argv) } Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); zkutil::ZooKeeper zk(argv[1]); LineReader lr({}, '\\'); From 70eaae38f2fe5c91d6a43b5166df623996826042 Mon Sep 17 00:00:00 2001 From: Ewout Date: Sat, 30 May 2020 21:44:22 +0200 Subject: [PATCH 125/150] Fix visitParamExtractRaw for strings with unbalanced { or [. Closes #11254 . --- src/Functions/visitParamExtractRaw.cpp | 13 ++++++++----- .../00539_functions_for_working_with_json.reference | 2 ++ .../00539_functions_for_working_with_json.sql | 2 ++ 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Functions/visitParamExtractRaw.cpp b/src/Functions/visitParamExtractRaw.cpp index 7a02d29e446..ef41e9bc900 100644 --- a/src/Functions/visitParamExtractRaw.cpp +++ b/src/Functions/visitParamExtractRaw.cpp @@ -22,6 +22,14 @@ struct ExtractRaw expects_end.pop_back(); current_expect_end = expects_end.empty() ? 0 : expects_end.back(); } + else if (current_expect_end == '"') + { + /// skip backslash + if (*pos == '\\' && pos + 1 < end && pos[1] == '"') + { + pos++; + } + } else { switch (*pos) @@ -38,11 +46,6 @@ struct ExtractRaw current_expect_end = '"'; expects_end.push_back(current_expect_end); break; - case '\\': - /// skip backslash - if (pos + 1 < end && pos[1] == '"') - pos++; - break; default: if (!current_expect_end && (*pos == ',' || *pos == '}')) { diff --git a/tests/queries/0_stateless/00539_functions_for_working_with_json.reference b/tests/queries/0_stateless/00539_functions_for_working_with_json.reference index ee7fb68b7c2..c0399f8ab2e 100644 --- a/tests/queries/0_stateless/00539_functions_for_working_with_json.reference +++ b/tests/queries/0_stateless/00539_functions_for_working_with_json.reference @@ -9,5 +9,7 @@ test"string "test_string" "test\\"string" "test\\"string" + "{" + "[" ["]", "2", "3"] {"nested" : [1,2,3]} diff --git a/tests/queries/0_stateless/00539_functions_for_working_with_json.sql b/tests/queries/0_stateless/00539_functions_for_working_with_json.sql index 8a4d1794293..514b5f2e5ea 100644 --- a/tests/queries/0_stateless/00539_functions_for_working_with_json.sql +++ b/tests/queries/0_stateless/00539_functions_for_working_with_json.sql @@ -11,5 +11,7 @@ SELECT visitParamExtractRaw('{"myparam":"test_string"}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": "test_string"}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": "test\\"string"}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": "test\\"string", "other":123}', 'myparam'); +SELECT visitParamExtractRaw('{"myparam": "{"}', 'myparam'); +SELECT visitParamExtractRaw('{"myparam": "["}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": ["]", "2", "3"], "other":123}', 'myparam'); SELECT visitParamExtractRaw('{"myparam": {"nested" : [1,2,3]}, "other":123}', 'myparam'); From 24a81f3d139dcd7a7dbc6a6af84cdddf00fd7c34 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 31 May 2020 11:23:25 +0300 Subject: [PATCH 126/150] Update common_meta.html --- website/templates/common_meta.html | 3 +++ 1 file changed, 3 insertions(+) diff --git a/website/templates/common_meta.html b/website/templates/common_meta.html index 8b65e67d780..212e0b089d5 100644 --- a/website/templates/common_meta.html +++ b/website/templates/common_meta.html @@ -22,6 +22,9 @@ +{% if config.extra.single_page or config.extra.version_prefix %} + +{% endif %} {% if config and page %} {% for code, name in config.extra.languages.items() %} From dc6638838dc664f3ed34ab6dd7ffd997e026fe3c Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 31 May 2020 11:23:41 +0300 Subject: [PATCH 127/150] Update common_meta.html --- website/templates/common_meta.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/common_meta.html b/website/templates/common_meta.html index 212e0b089d5..9eda30a14d9 100644 --- a/website/templates/common_meta.html +++ b/website/templates/common_meta.html @@ -23,7 +23,7 @@ {% if config.extra.single_page or config.extra.version_prefix %} - + {% endif %} {% if config and page %} From 4b4d3b559271b30dcced9f7cc1a1d9e74839b56c Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 31 May 2020 11:25:28 +0300 Subject: [PATCH 128/150] Update amp.html --- website/templates/docs/amp.html | 3 +++ 1 file changed, 3 insertions(+) diff --git a/website/templates/docs/amp.html b/website/templates/docs/amp.html index a60270d23d0..0cff614035f 100644 --- a/website/templates/docs/amp.html +++ b/website/templates/docs/amp.html @@ -7,6 +7,9 @@ {% include "templates/docs/ld_json.html" %} + {% if config.extra.single_page or config.extra.version_prefix %} + + {% endif %} From 154d045c1b0ec0a2ac4d490e75630f7edb0fffe0 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Sun, 31 May 2020 11:57:04 +0300 Subject: [PATCH 129/150] Update common_meta.html --- website/templates/common_meta.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/common_meta.html b/website/templates/common_meta.html index 9eda30a14d9..84bd93d5175 100644 --- a/website/templates/common_meta.html +++ b/website/templates/common_meta.html @@ -22,7 +22,7 @@ -{% if config.extra.single_page or config.extra.version_prefix %} +{% if config and (config.extra.single_page or config.extra.version_prefix) %} {% endif %} From b4446154040ec7cfea1d06fc935756981e7da794 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 16:47:58 +0300 Subject: [PATCH 130/150] Compatibility with Arcadia build --- base/common/logger_useful.h | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/base/common/logger_useful.h b/base/common/logger_useful.h index ec3933cf121..f760d59de45 100644 --- a/base/common/logger_useful.h +++ b/base/common/logger_useful.h @@ -49,3 +49,12 @@ namespace #define LOG_WARNING(logger, ...) LOG_IMPL(logger, DB::LogsLevel::warning, Poco::Message::PRIO_WARNING, __VA_ARGS__) #define LOG_ERROR(logger, ...) LOG_IMPL(logger, DB::LogsLevel::error, Poco::Message::PRIO_ERROR, __VA_ARGS__) #define LOG_FATAL(logger, ...) LOG_IMPL(logger, DB::LogsLevel::error, Poco::Message::PRIO_FATAL, __VA_ARGS__) + + +/// Compatibility for external projects. +#if defined(ARCADIA_BUILD) + using Poco::Logger; + using Poco::Message; + using DB::LogsLevel; + using DB::CurrentThread; +#endif From 669a15b943f35bed22194bc0a74e5595bf200d71 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 16:48:43 +0300 Subject: [PATCH 131/150] Whitespaces --- base/loggers/OwnFormattingChannel.cpp | 2 +- base/loggers/OwnSplitChannel.cpp | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/base/loggers/OwnFormattingChannel.cpp b/base/loggers/OwnFormattingChannel.cpp index 9fccc929364..f03d155bde7 100644 --- a/base/loggers/OwnFormattingChannel.cpp +++ b/base/loggers/OwnFormattingChannel.cpp @@ -4,6 +4,7 @@ namespace DB { + void OwnFormattingChannel::logExtended(const ExtendedLogMessage & msg) { if (pChannel && priority >= msg.base.getPriority()) @@ -28,5 +29,4 @@ void OwnFormattingChannel::log(const Poco::Message & msg) OwnFormattingChannel::~OwnFormattingChannel() = default; - } diff --git a/base/loggers/OwnSplitChannel.cpp b/base/loggers/OwnSplitChannel.cpp index 3f77b594fbb..22951d1b509 100644 --- a/base/loggers/OwnSplitChannel.cpp +++ b/base/loggers/OwnSplitChannel.cpp @@ -69,7 +69,6 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg) logs_queue->emplace(std::move(columns)); } - /// Also log to system.text_log table, if message is not too noisy auto text_log_max_priority_loaded = text_log_max_priority.load(std::memory_order_relaxed); if (text_log_max_priority_loaded && msg.getPriority() <= text_log_max_priority_loaded) From e3dc1d6734ebad8e6b47988b74a96dc4cec2b25b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 16:50:40 +0300 Subject: [PATCH 132/150] Fix build --- base/daemon/BaseDaemon.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 1ab140d2595..8c9ec77fd46 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -288,7 +288,7 @@ extern "C" void __sanitizer_set_death_callback(void (*)()); static void sanitizerDeathCallback() { - Logger * log = &Logger::get("BaseDaemon"); + Poco::Logger * log = &Poco::Logger::get("BaseDaemon"); StringRef query_id = CurrentThread::getQueryId(); /// This is signal safe. @@ -498,10 +498,10 @@ void debugIncreaseOOMScore() } catch (const Poco::Exception & e) { - LOG_WARNING(&Logger::root(), "Failed to adjust OOM score: '{}'.", e.displayText()); + LOG_WARNING(&Poco::Logger::root(), "Failed to adjust OOM score: '{}'.", e.displayText()); return; } - LOG_INFO(&Logger::root(), "Set OOM score adjustment to {}", new_score); + LOG_INFO(&Poco::Logger::root(), "Set OOM score adjustment to {}", new_score); } #else void debugIncreaseOOMScore() {} From d2eb58142e3984e3ae2e9a5c45a4eac4948f460a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 16:51:37 +0300 Subject: [PATCH 133/150] Fix build --- base/daemon/BaseDaemon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 8c9ec77fd46..2a36777218e 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -290,7 +290,7 @@ static void sanitizerDeathCallback() { Poco::Logger * log = &Poco::Logger::get("BaseDaemon"); - StringRef query_id = CurrentThread::getQueryId(); /// This is signal safe. + StringRef query_id = DB::CurrentThread::getQueryId(); /// This is signal safe. { std::stringstream message; From 2529df016ac3b50497b4c25394a260d3772fb21d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 17:08:09 +0300 Subject: [PATCH 134/150] Disable flacky test #8840 --- tests/integration/test_ttl_move/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index fc5c6725148..ae2af34c35b 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -834,6 +834,7 @@ def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): finally: node1.query("DROP TABLE IF EXISTS {name}".format(name=name)) +@pytest.mark.skip(reason="Flacky test") @pytest.mark.parametrize("name,positive", [ ("test_double_move_while_select_negative", 0), ("test_double_move_while_select_positive", 1), From 26395709a1cf97ec5c46290bd24e61aa99f7e4fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 17:24:01 +0300 Subject: [PATCH 135/150] Regenerate ya.make to prepare for #10678 --- src/DataTypes/ya.make | 3 ++- src/DataTypes/ya.make.in | 12 ++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 src/DataTypes/ya.make.in diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index c3a1e452d0d..c4e7ac8465a 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -13,8 +13,8 @@ SRCS( DataTypeCustomSimpleAggregateFunction.cpp DataTypeCustomSimpleTextSerialization.cpp DataTypeDate.cpp - DataTypeDateTime.cpp DataTypeDateTime64.cpp + DataTypeDateTime.cpp DataTypeDecimalBase.cpp DataTypeEnum.cpp DataTypeFactory.cpp @@ -36,6 +36,7 @@ SRCS( getMostSubtype.cpp IDataType.cpp NestedUtils.cpp + ) END() diff --git a/src/DataTypes/ya.make.in b/src/DataTypes/ya.make.in new file mode 100644 index 00000000000..39cbdefe361 --- /dev/null +++ b/src/DataTypes/ya.make.in @@ -0,0 +1,12 @@ +LIBRARY() + +PEERDIR( + clickhouse/src/Common + clickhouse/src/Formats +) + +SRCS( + +) + +END() From de2f7a0a9dd08454cb9bd10d38dcd0237de75c4c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 17:26:00 +0300 Subject: [PATCH 136/150] Regenerate ya.make after #10678 --- src/DataTypes/ya.make | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index 594227357c5..7d507a2f611 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -9,8 +9,8 @@ SRCS( convertMySQLDataType.cpp DataTypeAggregateFunction.cpp DataTypeArray.cpp + DataTypeCustomGeo.cpp DataTypeCustomIPv4AndIPv6.cpp - DataTypeCustomPoint.cpp DataTypeCustomSimpleAggregateFunction.cpp DataTypeCustomSimpleTextSerialization.cpp DataTypeDate.cpp From f47320cb96510e1916ddb90912f8c9f2eacd7c99 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 17:28:44 +0300 Subject: [PATCH 137/150] Added clarification #10678 --- src/DataTypes/ya.make | 1 + src/Dictionaries/ya.make | 1 + src/Functions/ya.make | 1 + utils/generate-ya-make/generate-ya-make.sh | 3 ++- 4 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index 7d507a2f611..82e9baf76f2 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -1,3 +1,4 @@ +# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. LIBRARY() PEERDIR( diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index e30a22a372c..12983b9527a 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -1,3 +1,4 @@ +# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. LIBRARY() PEERDIR( diff --git a/src/Functions/ya.make b/src/Functions/ya.make index bbbf13d360b..70c42dd5af7 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -1,3 +1,4 @@ +# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. LIBRARY() CFLAGS( diff --git a/utils/generate-ya-make/generate-ya-make.sh b/utils/generate-ya-make/generate-ya-make.sh index ec4511f56c1..84be2177386 100755 --- a/utils/generate-ya-make/generate-ya-make.sh +++ b/utils/generate-ya-make/generate-ya-make.sh @@ -8,5 +8,6 @@ ROOT_PATH=$(git rev-parse --show-toplevel) EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing' find "${ROOT_PATH}" -name 'ya.make.in' | while read path; do - (cd $(dirname "${path}") && perl -pne 's/<\?(.+?)\?>/`$1`/e' < "${path}" > "${path/.in/}") + echo "# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it." > "${path/.in/}" + (cd $(dirname "${path}") && perl -pne 's/<\?(.+?)\?>/`$1`/e' < "${path}" >> "${path/.in/}") done From 85537d7b94f4442fc02d1ab438cd057da45e4716 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 29 May 2020 19:57:53 +0300 Subject: [PATCH 138/150] Miscellaneous changes for SystemLogs --- src/Interpreters/SystemLog.cpp | 12 ++++++------ src/Interpreters/loadMetadata.cpp | 1 - 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index e7e65c4f19e..229643b7e02 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -76,12 +76,6 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi text_log = createSystemLog(global_context, "system", "text_log", config, "text_log"); metric_log = createSystemLog(global_context, "system", "metric_log", config, "metric_log"); - if (metric_log) - { - size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds"); - metric_log->startCollectMetric(collect_interval_milliseconds); - } - if (query_log) logs.emplace_back(query_log.get()); if (query_thread_log) @@ -112,6 +106,12 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi shutdown(); throw; } + + if (metric_log) + { + size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds"); + metric_log->startCollectMetric(collect_interval_milliseconds); + } } diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index bf1f64f0a77..bf5701e1be4 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -48,7 +48,6 @@ static void loadDatabase( const String & database_path, bool force_restore_data) { - String database_attach_query; String database_metadata_file = database_path + ".sql"; From 293241cc22bdf0c6782957b52c60bfd0644c36c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 17:57:29 +0300 Subject: [PATCH 139/150] Better check for hung queries in clickhouse-test --- tests/clickhouse-test | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 60390c999ab..219546c92f1 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -502,7 +502,14 @@ def main(args): total_tests_run += tests_n if args.hung_check: - processlist = get_processlist(args.client) + + # Some queries may execute in background for some time after test was finished. This is normal. + for n in range(1, 60): + processlist = get_processlist(args.client) + if not processlist: + break + sleep(1) + if processlist: print(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"])) print(processlist) From e1202759c550d88e193d029efbdb59634e552be9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 18:01:59 +0300 Subject: [PATCH 140/150] Fix style #11318 --- src/Functions/visitParamExtractRaw.cpp | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Functions/visitParamExtractRaw.cpp b/src/Functions/visitParamExtractRaw.cpp index ef41e9bc900..add882f003f 100644 --- a/src/Functions/visitParamExtractRaw.cpp +++ b/src/Functions/visitParamExtractRaw.cpp @@ -26,9 +26,7 @@ struct ExtractRaw { /// skip backslash if (*pos == '\\' && pos + 1 < end && pos[1] == '"') - { - pos++; - } + ++pos; } else { From f7c174c737ef74bef26180a1f528a7a9519a6136 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 May 2020 18:23:21 +0300 Subject: [PATCH 141/150] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f32bf8dde21..abdcfd70941 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -17,7 +17,7 @@ * Fixed crash in `SELECT count(notNullIn(NULL, []))`. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fixed the hang which was happening sometimes during `DROP` of `Kafka` table engine. (or during server restarts). [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). * Fixed the impossibility of executing multiple `ALTER RENAME` like `a TO b, c TO a`. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). -* Fixed possible race which could happen when you get result from aggregate function state from multiple thread for the same column. The only way it can happen is when you use `finalizeAggregation` function while reading from table with `Memory` engine which stores `AggregateFunction` state for `quanite*` function. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible race which could happen when you get result from aggregate function state from multiple thread for the same column. The only way it can happen is when you use `finalizeAggregation` function while reading from table with `Memory` engine which stores `AggregateFunction` state for `quantile*` function. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fixed backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). * Fixed `SIGSEGV` in `StringHashTable` if such a key does not exist. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). * Fixed `WATCH` hangs after `LiveView` table was dropped from database with `Atomic` engine. [#10859](https://github.com/ClickHouse/ClickHouse/pull/10859) ([tavplubix](https://github.com/tavplubix)). From 1396b91f69f1931b55ef2f3e81b872f08dccc02d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 May 2020 18:27:21 +0300 Subject: [PATCH 142/150] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index abdcfd70941..6a4666f08bb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -40,7 +40,7 @@ * Implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). * Disable `GROUP BY` sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for `WITH ROLLUP/CUBE/TOTALS`. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). * Fixed [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). -* Added failing tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Added backward compatibility for create bloom filter index. [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). ### ClickHouse release v20.4.2.9, 2020-05-12 From 9416e82b752b52a05ec2f794e651c57273317a23 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 May 2020 19:06:59 +0300 Subject: [PATCH 143/150] Update MergeTreeDataMergerMutator.cpp --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index cf187310ae0..679d73286a3 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -799,7 +799,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merged_stream = std::make_shared(merged_stream); } - MergedBlockOutputStream to{ new_data_part, merging_columns, From 07c14e12dd193c29c06446d535c7baa557d481a9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 19:53:30 +0300 Subject: [PATCH 144/150] Fix build after merge of #11162 with master --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 264e0f34075..f906b329d2d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -717,7 +717,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor if (data.hasSortingKey()) { - auto expr = std::make_shared(pipe.getHeader(), data.sorting_key_expr); + auto expr = std::make_shared(pipe.getHeader(), data.getSortingKey().expression); pipe.addSimpleTransform(std::move(expr)); } From 585a5b6f3ea0b3a56d294a97bf4dc9752d968fa8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 21:49:19 +0300 Subject: [PATCH 145/150] Whitespace --- programs/client/Client.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 878eaede2fe..930e55a06f8 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1175,7 +1175,7 @@ private: /// Poll for changes after a cancellation check, otherwise it never reached /// because of progress updates from server. if (connection->poll(poll_interval)) - break; + break; } if (!receiveAndProcessPacket()) From 463d2cf836e344b4e049faa2fbfbaa7db450fd51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 22:22:18 +0300 Subject: [PATCH 146/150] Improve performance of UTF8::computeWidth --- src/Common/StringUtils/StringUtils.h | 6 ++++++ src/Common/UTF8Helpers.cpp | 9 +++++++++ 2 files changed, 15 insertions(+) diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils/StringUtils.h index 390015f3d2b..bac33bdb508 100644 --- a/src/Common/StringUtils/StringUtils.h +++ b/src/Common/StringUtils/StringUtils.h @@ -116,6 +116,12 @@ inline bool isControlASCII(char c) return static_cast(c) <= 31; } +inline bool isPrintableASCII(char c) +{ + uint8_t uc = c; + return uc >= 32 && uc <= 126; /// 127 is ASCII DEL. +} + /// Works assuming isAlphaASCII. inline char toLowerIfAlphaASCII(char c) { diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index d393ee2328b..182365d3227 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -1,7 +1,9 @@ #include +#include #include + namespace DB { namespace UTF8 @@ -94,6 +96,13 @@ size_t computeWidth(const UInt8 * data, size_t size, size_t prefix) noexcept size_t rollback = 0; for (size_t i = 0; i < size; ++i) { + /// Quickly skip regular ASCII + while (i < size && isPrintableASCII(data[i])) + { + ++width; + ++i; + } + switch (decoder.decode(data[i])) { case UTF8Decoder::REJECT: From ad14025b5e88b8a3152f8096dff5b1664b65950c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 May 2020 22:47:39 +0300 Subject: [PATCH 147/150] Further improvement --- src/Common/UTF8Helpers.cpp | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/src/Common/UTF8Helpers.cpp b/src/Common/UTF8Helpers.cpp index 182365d3227..3cc29371b64 100644 --- a/src/Common/UTF8Helpers.cpp +++ b/src/Common/UTF8Helpers.cpp @@ -97,6 +97,35 @@ size_t computeWidth(const UInt8 * data, size_t size, size_t prefix) noexcept for (size_t i = 0; i < size; ++i) { /// Quickly skip regular ASCII + +#if defined(__SSE2__) + const auto lower_bound = _mm_set1_epi8(32); + const auto upper_bound = _mm_set1_epi8(126); + + while (i + 15 < size) + { + __m128i bytes = _mm_loadu_si128(reinterpret_cast(&data[i])); + + const uint16_t non_regular_width_mask = _mm_movemask_epi8( + _mm_or_si128( + _mm_cmplt_epi8(bytes, lower_bound), + _mm_cmpgt_epi8(bytes, upper_bound))); + + if (non_regular_width_mask) + { + auto num_regular_chars = __builtin_ctz(non_regular_width_mask); + width += num_regular_chars; + i += num_regular_chars; + break; + } + else + { + i += 16; + width += 16; + } + } +#endif + while (i < size && isPrintableASCII(data[i])) { ++width; From 7fa1366a4a818166f962a524629075c68c5fe182 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Jun 2020 01:25:37 +0300 Subject: [PATCH 148/150] Fix build after merge #11230 --- ...{PocoHttpClient.cpp => PocoHTTPClient.cpp} | 34 ++++++++++--------- .../S3/{PocoHttpClient.h => PocoHTTPClient.h} | 6 ++-- ...tFactory.cpp => PocoHTTPClientFactory.cpp} | 14 ++++---- ...lientFactory.h => PocoHTTPClientFactory.h} | 4 +-- src/IO/S3Common.cpp | 10 +++--- 5 files changed, 35 insertions(+), 33 deletions(-) rename src/IO/S3/{PocoHttpClient.cpp => PocoHTTPClient.cpp} (83%) rename src/IO/S3/{PocoHttpClient.h => PocoHTTPClient.h} (88%) rename src/IO/S3/{PocoHttpClientFactory.cpp => PocoHTTPClientFactory.cpp} (67%) rename src/IO/S3/{PocoHttpClientFactory.h => PocoHTTPClientFactory.h} (86%) diff --git a/src/IO/S3/PocoHttpClient.cpp b/src/IO/S3/PocoHTTPClient.cpp similarity index 83% rename from src/IO/S3/PocoHttpClient.cpp rename to src/IO/S3/PocoHTTPClient.cpp index f7b71c956a7..b8de483a5a8 100644 --- a/src/IO/S3/PocoHttpClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -1,4 +1,4 @@ -#include "PocoHttpClient.h" +#include "PocoHTTPClient.h" #include #include @@ -14,7 +14,7 @@ namespace DB::S3 { -PocoHttpClient::PocoHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration) +PocoHTTPClient::PocoHTTPClient(const Aws::Client::ClientConfiguration & clientConfiguration) : per_request_configuration(clientConfiguration.perRequestConfiguration) , timeouts(ConnectionTimeouts( Poco::Timespan(clientConfiguration.connectTimeoutMs * 1000), /// connection timeout. @@ -24,34 +24,36 @@ PocoHttpClient::PocoHttpClient(const Aws::Client::ClientConfiguration & clientCo { } -std::shared_ptr PocoHttpClient::MakeRequest( +std::shared_ptr PocoHTTPClient::MakeRequest( Aws::Http::HttpRequest & request, Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const { - auto response = Aws::MakeShared("PocoHttpClient", request); + auto response = Aws::MakeShared("PocoHTTPClient", request); MakeRequestInternal(request, response, readLimiter, writeLimiter); return response; } -std::shared_ptr PocoHttpClient::MakeRequest( +std::shared_ptr PocoHTTPClient::MakeRequest( const std::shared_ptr & request, Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, Aws::Utils::RateLimits::RateLimiterInterface * writeLimiter) const { - auto response = Aws::MakeShared("PocoHttpClient", request); + auto response = Aws::MakeShared("PocoHTTPClient", request); MakeRequestInternal(*request, response, readLimiter, writeLimiter); return response; } -void PocoHttpClient::MakeRequestInternal( +void PocoHTTPClient::MakeRequestInternal( Aws::Http::HttpRequest & request, std::shared_ptr & response, Aws::Utils::RateLimits::RateLimiterInterface *, Aws::Utils::RateLimits::RateLimiterInterface *) const { + Poco::Logger * log = &Poco::Logger::get("AWSClient"); + auto uri = request.GetUri().GetURIString(); - LOG_DEBUG(&Logger::get("AWSClient"), "Make request to: {}", uri); + LOG_DEBUG(log, "Make request to: {}", uri); const int MAX_REDIRECT_ATTEMPTS = 10; try @@ -102,27 +104,27 @@ void PocoHttpClient::MakeRequestInternal( if (request.GetContentBody()) { - LOG_DEBUG(&Logger::get("AWSClient"), "Writing request body..."); + LOG_DEBUG(log, "Writing request body."); if (attempt > 0) /// rewind content body buffer. { request.GetContentBody()->clear(); request.GetContentBody()->seekg(0); } auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); - LOG_DEBUG(&Logger::get("AWSClient"), "Written {} bytes to request body", size); + LOG_DEBUG(log, "Written {} bytes to request body", size); } - LOG_DEBUG(&Logger::get("AWSClient"), "Receiving response..."); + LOG_DEBUG(log, "Receiving response..."); auto & response_body_stream = session->receiveResponse(poco_response); int status_code = static_cast(poco_response.getStatus()); - LOG_DEBUG(&Logger::get("AWSClient"), "Response status: {}, {}", status_code, poco_response.getReason()); + LOG_DEBUG(log, "Response status: {}, {}", status_code, poco_response.getReason()); if (poco_response.getStatus() == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) { auto location = poco_response.get("location"); uri = location; - LOG_DEBUG(&Logger::get("AWSClient"), "Redirecting request to new location: {}", location); + LOG_DEBUG(log, "Redirecting request to new location: {}", location); continue; } @@ -134,9 +136,9 @@ void PocoHttpClient::MakeRequestInternal( for (const auto & [header_name, header_value] : poco_response) { response->AddHeader(header_name, header_value); - headers_ss << header_name << " : " << header_value << "; "; + headers_ss << header_name << ": " << header_value << "; "; } - LOG_DEBUG(&Logger::get("AWSClient"), "Received headers: {}", headers_ss.str()); + LOG_DEBUG(log, "Received headers: {}", headers_ss.str()); if (status_code >= 300) { @@ -155,7 +157,7 @@ void PocoHttpClient::MakeRequestInternal( } catch (...) { - tryLogCurrentException(&Logger::get("AWSClient"), "Failed to make request to: " + uri); + tryLogCurrentException(log, fmt::format("Failed to make request to: {}", uri)); response->SetClientErrorType(Aws::Client::CoreErrors::NETWORK_CONNECTION); response->SetClientErrorMessage(getCurrentExceptionMessage(false)); } diff --git a/src/IO/S3/PocoHttpClient.h b/src/IO/S3/PocoHTTPClient.h similarity index 88% rename from src/IO/S3/PocoHttpClient.h rename to src/IO/S3/PocoHTTPClient.h index 7458c692a0e..203ad94e86b 100644 --- a/src/IO/S3/PocoHttpClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -11,11 +11,11 @@ class StandardHttpResponse; namespace DB::S3 { -class PocoHttpClient : public Aws::Http::HttpClient +class PocoHTTPClient : public Aws::Http::HttpClient { public: - explicit PocoHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration); - ~PocoHttpClient() override = default; + explicit PocoHTTPClient(const Aws::Client::ClientConfiguration & clientConfiguration); + ~PocoHTTPClient() override = default; std::shared_ptr MakeRequest( Aws::Http::HttpRequest & request, Aws::Utils::RateLimits::RateLimiterInterface * readLimiter, diff --git a/src/IO/S3/PocoHttpClientFactory.cpp b/src/IO/S3/PocoHTTPClientFactory.cpp similarity index 67% rename from src/IO/S3/PocoHttpClientFactory.cpp rename to src/IO/S3/PocoHTTPClientFactory.cpp index 96a73e1d001..033ad4af37c 100644 --- a/src/IO/S3/PocoHttpClientFactory.cpp +++ b/src/IO/S3/PocoHTTPClientFactory.cpp @@ -1,6 +1,6 @@ -#include "PocoHttpClientFactory.h" +#include "PocoHTTPClientFactory.h" -#include +#include #include #include #include @@ -9,21 +9,21 @@ namespace DB::S3 { std::shared_ptr -PocoHttpClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration) const +PocoHTTPClientFactory::CreateHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration) const { - return std::make_shared(clientConfiguration); + return std::make_shared(clientConfiguration); } -std::shared_ptr PocoHttpClientFactory::CreateHttpRequest( +std::shared_ptr PocoHTTPClientFactory::CreateHttpRequest( const Aws::String & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const { return CreateHttpRequest(Aws::Http::URI(uri), method, streamFactory); } -std::shared_ptr PocoHttpClientFactory::CreateHttpRequest( +std::shared_ptr PocoHTTPClientFactory::CreateHttpRequest( const Aws::Http::URI & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const { - auto request = Aws::MakeShared("PocoHttpClientFactory", uri, method); + auto request = Aws::MakeShared("PocoHTTPClientFactory", uri, method); request->SetResponseStreamFactory(streamFactory); return request; diff --git a/src/IO/S3/PocoHttpClientFactory.h b/src/IO/S3/PocoHTTPClientFactory.h similarity index 86% rename from src/IO/S3/PocoHttpClientFactory.h rename to src/IO/S3/PocoHTTPClientFactory.h index ac73a0356ff..ac586289113 100644 --- a/src/IO/S3/PocoHttpClientFactory.h +++ b/src/IO/S3/PocoHTTPClientFactory.h @@ -11,10 +11,10 @@ namespace Aws::Http namespace DB::S3 { -class PocoHttpClientFactory : public Aws::Http::HttpClientFactory +class PocoHTTPClientFactory : public Aws::Http::HttpClientFactory { public: - ~PocoHttpClientFactory() override = default; + ~PocoHTTPClientFactory() override = default; [[nodiscard]] std::shared_ptr CreateHttpClient(const Aws::Client::ClientConfiguration & clientConfiguration) const override; [[nodiscard]] std::shared_ptr CreateHttpRequest(const Aws::String & uri, Aws::Http::HttpMethod method, const Aws::IOStreamFactory & streamFactory) const override; diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 9aa218b254a..1539b3c7025 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -10,10 +10,10 @@ # include # include # include -# include -# include -# include -# include +# include +# include +# include +# include # include # include # include @@ -76,7 +76,7 @@ namespace S3 aws_options = Aws::SDKOptions{}; Aws::InitAPI(aws_options); Aws::Utils::Logging::InitializeAWSLogging(std::make_shared()); - Aws::Http::SetHttpClientFactory(std::make_shared()); + Aws::Http::SetHttpClientFactory(std::make_shared()); } ClientFactory::~ClientFactory() From 9028a25f3c549b85a08bdd82359da7861aa5973c Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Mon, 1 Jun 2020 12:38:46 +0300 Subject: [PATCH 149/150] more types in ASOF JOIN (#11301) --- src/Columns/ColumnDecimal.cpp | 11 ++ src/Columns/ColumnDecimal.h | 2 + src/Columns/ColumnVector.cpp | 27 ++++ src/Columns/ColumnVector.h | 2 + src/Interpreters/HashJoin.h | 4 +- src/Interpreters/RowRefs.cpp | 124 ++++++++++-------- src/Interpreters/RowRefs.h | 25 ++-- .../01139_asof_join_types.reference | 13 ++ .../0_stateless/01139_asof_join_types.sql | 18 +++ 9 files changed, 150 insertions(+), 76 deletions(-) create mode 100644 tests/queries/0_stateless/01139_asof_join_types.reference create mode 100644 tests/queries/0_stateless/01139_asof_join_types.sql diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 3e6fb833b56..1c238cc6458 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -333,6 +333,17 @@ void ColumnDecimal::getExtremes(Field & min, Field & max) const max = NearestFieldType(cur_max, scale); } +TypeIndex columnDecimalDataType(const IColumn * column) +{ + if (checkColumn>(column)) + return TypeIndex::Decimal32; + else if (checkColumn>(column)) + return TypeIndex::Decimal64; + else if (checkColumn>(column)) + return TypeIndex::Decimal128; + return TypeIndex::Nothing; +} + template class ColumnDecimal; template class ColumnDecimal; template class ColumnDecimal; diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 86357dc8be7..16c6a47c30a 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -197,4 +197,6 @@ ColumnPtr ColumnDecimal::indexImpl(const PaddedPODArray & indexes, size return res; } +TypeIndex columnDecimalDataType(const IColumn * column); + } diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index c4f2c6d8705..95efd0dedad 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -517,6 +517,33 @@ void ColumnVector::getExtremes(Field & min, Field & max) const max = NearestFieldType(cur_max); } +TypeIndex columnVectorDataType(const IColumn * column) +{ + if (checkColumn>(column)) + return TypeIndex::UInt8; + else if (checkColumn>(column)) + return TypeIndex::UInt16; + else if (checkColumn>(column)) + return TypeIndex::UInt32; + else if (checkColumn>(column)) + return TypeIndex::UInt64; + else if (checkColumn>(column)) + return TypeIndex::Int8; + else if (checkColumn>(column)) + return TypeIndex::Int16; + else if (checkColumn>(column)) + return TypeIndex::Int32; + else if (checkColumn>(column)) + return TypeIndex::Int64; + else if (checkColumn>(column)) + return TypeIndex::Int128; + else if (checkColumn>(column)) + return TypeIndex::Float32; + else if (checkColumn>(column)) + return TypeIndex::Float64; + return TypeIndex::Nothing; +} + /// Explicit template instantiations - to avoid code bloat in headers. template class ColumnVector; template class ColumnVector; diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index a6105034f1a..5e934b42df0 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -320,4 +320,6 @@ ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_ return res; } +TypeIndex columnVectorDataType(const IColumn * column); + } diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 9d4e0907f66..7c06ec13f02 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -192,7 +192,7 @@ public: ASTTableJoin::Kind getKind() const { return kind; } ASTTableJoin::Strictness getStrictness() const { return strictness; } - AsofRowRefs::Type getAsofType() const { return *asof_type; } + TypeIndex getAsofType() const { return *asof_type; } ASOF::Inequality getAsofInequality() const { return asof_inequality; } bool anyTakeLastRow() const { return any_take_last_row; } @@ -344,7 +344,7 @@ private: bool nullable_right_side; /// In case of LEFT and FULL joins, if use_nulls, convert right-side columns to Nullable. bool nullable_left_side; /// In case of RIGHT and FULL joins, if use_nulls, convert left-side columns to Nullable. bool any_take_last_row; /// Overwrite existing values when encountering the same key again - std::optional asof_type; + std::optional asof_type; ASOF::Inequality asof_inequality; /// Right table data. StorageJoin shares it between many Join objects. diff --git a/src/Interpreters/RowRefs.cpp b/src/Interpreters/RowRefs.cpp index 5d1f1217241..e10f8bb2ea7 100644 --- a/src/Interpreters/RowRefs.cpp +++ b/src/Interpreters/RowRefs.cpp @@ -17,19 +17,25 @@ namespace /// maps enum values to types template -void callWithType(AsofRowRefs::Type which, F && f) +void callWithType(TypeIndex which, F && f) { switch (which) { - case AsofRowRefs::Type::keyu32: return f(UInt32()); - case AsofRowRefs::Type::keyu64: return f(UInt64()); - case AsofRowRefs::Type::keyi32: return f(Int32()); - case AsofRowRefs::Type::keyi64: return f(Int64()); - case AsofRowRefs::Type::keyf32: return f(Float32()); - case AsofRowRefs::Type::keyf64: return f(Float64()); - case AsofRowRefs::Type::keyDecimal32: return f(Decimal32()); - case AsofRowRefs::Type::keyDecimal64: return f(Decimal64()); - case AsofRowRefs::Type::keyDecimal128: return f(Decimal128()); + case TypeIndex::UInt8: return f(UInt8()); + case TypeIndex::UInt16: return f(UInt16()); + case TypeIndex::UInt32: return f(UInt32()); + case TypeIndex::UInt64: return f(UInt64()); + case TypeIndex::Int8: return f(Int8()); + case TypeIndex::Int16: return f(Int16()); + case TypeIndex::Int32: return f(Int32()); + case TypeIndex::Int64: return f(Int64()); + case TypeIndex::Float32: return f(Float32()); + case TypeIndex::Float64: return f(Float64()); + case TypeIndex::Decimal32: return f(Decimal32()); + case TypeIndex::Decimal64: return f(Decimal64()); + case TypeIndex::Decimal128: return f(Decimal128()); + default: + break; } __builtin_unreachable(); @@ -38,7 +44,7 @@ void callWithType(AsofRowRefs::Type which, F && f) } -AsofRowRefs::AsofRowRefs(Type type) +AsofRowRefs::AsofRowRefs(TypeIndex type) { auto call = [&](const auto & t) { @@ -50,7 +56,7 @@ AsofRowRefs::AsofRowRefs(Type type) callWithType(type, call); } -void AsofRowRefs::insert(Type type, const IColumn * asof_column, const Block * block, size_t row_num) +void AsofRowRefs::insert(TypeIndex type, const IColumn * asof_column, const Block * block, size_t row_num) { auto call = [&](const auto & t) { @@ -68,7 +74,7 @@ void AsofRowRefs::insert(Type type, const IColumn * asof_column, const Block * b callWithType(type, call); } -const RowRef * AsofRowRefs::findAsof(Type type, ASOF::Inequality inequality, const IColumn * asof_column, size_t row_num) const +const RowRef * AsofRowRefs::findAsof(TypeIndex type, ASOF::Inequality inequality, const IColumn * asof_column, size_t row_num) const { const RowRef * out = nullptr; @@ -96,52 +102,56 @@ const RowRef * AsofRowRefs::findAsof(Type type, ASOF::Inequality inequality, con return out; } -std::optional AsofRowRefs::getTypeSize(const IColumn * asof_column, size_t & size) +std::optional AsofRowRefs::getTypeSize(const IColumn * asof_column, size_t & size) { - if (typeid_cast *>(asof_column)) + TypeIndex idx = columnVectorDataType(asof_column); + if (idx == TypeIndex::Nothing) + idx = columnDecimalDataType(asof_column); + + switch (idx) { - size = sizeof(UInt32); - return Type::keyu32; - } - else if (typeid_cast *>(asof_column)) - { - size = sizeof(UInt64); - return Type::keyu64; - } - else if (typeid_cast *>(asof_column)) - { - size = sizeof(Int32); - return Type::keyi32; - } - else if (typeid_cast *>(asof_column)) - { - size = sizeof(Int64); - return Type::keyi64; - } - else if (typeid_cast *>(asof_column)) - { - size = sizeof(Float32); - return Type::keyf32; - } - else if (typeid_cast *>(asof_column)) - { - size = sizeof(Float64); - return Type::keyf64; - } - else if (typeid_cast *>(asof_column)) - { - size = sizeof(Decimal32); - return Type::keyDecimal32; - } - else if (typeid_cast *>(asof_column)) - { - size = sizeof(Decimal64); - return Type::keyDecimal64; - } - else if (typeid_cast *>(asof_column)) - { - size = sizeof(Decimal128); - return Type::keyDecimal128; + case TypeIndex::UInt8: + size = sizeof(UInt8); + return idx; + case TypeIndex::UInt16: + size = sizeof(UInt16); + return idx; + case TypeIndex::UInt32: + size = sizeof(UInt32); + return idx; + case TypeIndex::UInt64: + size = sizeof(UInt64); + return idx; + case TypeIndex::Int8: + size = sizeof(Int8); + return idx; + case TypeIndex::Int16: + size = sizeof(Int16); + return idx; + case TypeIndex::Int32: + size = sizeof(Int32); + return idx; + case TypeIndex::Int64: + size = sizeof(Int64); + return idx; + //case TypeIndex::Int128: + case TypeIndex::Float32: + size = sizeof(Float32); + return idx; + case TypeIndex::Float64: + size = sizeof(Float64); + return idx; + case TypeIndex::Decimal32: + size = sizeof(Decimal32); + return idx; + case TypeIndex::Decimal64: + size = sizeof(Decimal64); + return idx; + case TypeIndex::Decimal128: + size = sizeof(Decimal128); + return idx; + default: + break; } size = 0; diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index 65ae16e2b49..e8231b1c233 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -216,8 +216,12 @@ public: }; using Lookups = std::variant< + Entry::LookupPtr, + Entry::LookupPtr, Entry::LookupPtr, Entry::LookupPtr, + Entry::LookupPtr, + Entry::LookupPtr, Entry::LookupPtr, Entry::LookupPtr, Entry::LookupPtr, @@ -226,29 +230,16 @@ public: Entry::LookupPtr, Entry::LookupPtr>; - enum class Type - { - keyu32, - keyu64, - keyi32, - keyi64, - keyf32, - keyf64, - keyDecimal32, - keyDecimal64, - keyDecimal128, - }; - AsofRowRefs() {} - AsofRowRefs(Type t); + AsofRowRefs(TypeIndex t); - static std::optional getTypeSize(const IColumn * asof_column, size_t & type_size); + static std::optional getTypeSize(const IColumn * asof_column, size_t & type_size); // This will be synchronized by the rwlock mutex in Join.h - void insert(Type type, const IColumn * asof_column, const Block * block, size_t row_num); + void insert(TypeIndex type, const IColumn * asof_column, const Block * block, size_t row_num); // This will internally synchronize - const RowRef * findAsof(Type type, ASOF::Inequality inequality, const IColumn * asof_column, size_t row_num) const; + const RowRef * findAsof(TypeIndex type, ASOF::Inequality inequality, const IColumn * asof_column, size_t row_num) const; private: // Lookups can be stored in a HashTable because it is memmovable diff --git a/tests/queries/0_stateless/01139_asof_join_types.reference b/tests/queries/0_stateless/01139_asof_join_types.reference new file mode 100644 index 00000000000..102064ce772 --- /dev/null +++ b/tests/queries/0_stateless/01139_asof_join_types.reference @@ -0,0 +1,13 @@ +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 0000-00-00 +0 0000-00-00 00:00:00 diff --git a/tests/queries/0_stateless/01139_asof_join_types.sql b/tests/queries/0_stateless/01139_asof_join_types.sql new file mode 100644 index 00000000000..45edf8ff06b --- /dev/null +++ b/tests/queries/0_stateless/01139_asof_join_types.sql @@ -0,0 +1,18 @@ +select * from (select 0 as k, toInt8(1) as v) t1 asof join (select 0 as k, toInt8(0) as v) t2 using(k, v); +select * from (select 0 as k, toInt16(1) as v) t1 asof join (select 0 as k, toInt16(0) as v) t2 using(k, v); +select * from (select 0 as k, toInt32(1) as v) t1 asof join (select 0 as k, toInt32(0) as v) t2 using(k, v); +select * from (select 0 as k, toInt64(1) as v) t1 asof join (select 0 as k, toInt64(0) as v) t2 using(k, v); + +select * from (select 0 as k, toUInt8(1) as v) t1 asof join (select 0 as k, toUInt8(0) as v) t2 using(k, v); +select * from (select 0 as k, toUInt16(1) as v) t1 asof join (select 0 as k, toUInt16(0) as v) t2 using(k, v); +select * from (select 0 as k, toUInt32(1) as v) t1 asof join (select 0 as k, toUInt32(0) as v) t2 using(k, v); +select * from (select 0 as k, toUInt64(1) as v) t1 asof join (select 0 as k, toUInt64(0) as v) t2 using(k, v); + +select * from (select 0 as k, toDecimal32(1, 0) as v) t1 asof join (select 0 as k, toDecimal32(0, 0) as v) t2 using(k, v); +select * from (select 0 as k, toDecimal64(1, 0) as v) t1 asof join (select 0 as k, toDecimal64(0, 0) as v) t2 using(k, v); +select * from (select 0 as k, toDecimal128(1, 0) as v) t1 asof join (select 0 as k, toDecimal128(0, 0) as v) t2 using(k, v); + +select * from (select 0 as k, toDate(0) as v) t1 asof join (select 0 as k, toDate(0) as v) t2 using(k, v); +select * from (select 0 as k, toDateTime(0) as v) t1 asof join (select 0 as k, toDateTime(0) as v) t2 using(k, v); + +select * from (select 0 as k, 'x' as v) t1 asof join (select 0 as k, 'x' as v) t2 using(k, v); -- { serverError 169 } From ca3fb27b093d89126f554eb3ba52be7908f08b85 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 1 Jun 2020 14:08:14 +0300 Subject: [PATCH 150/150] Report dictionary name on dictionary load errors. --- src/Databases/DatabaseDictionary.cpp | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Databases/DatabaseDictionary.cpp b/src/Databases/DatabaseDictionary.cpp index 67d33d7bfd7..6c5433cab38 100644 --- a/src/Databases/DatabaseDictionary.cpp +++ b/src/Databases/DatabaseDictionary.cpp @@ -23,10 +23,20 @@ namespace { StoragePtr createStorageDictionary(const String & database_name, const ExternalLoader::LoadResult & load_result) { - if (!load_result.config) - return nullptr; - DictionaryStructure dictionary_structure = ExternalDictionariesLoader::getDictionaryStructure(*load_result.config); - return StorageDictionary::create(StorageID(database_name, load_result.name), load_result.name, dictionary_structure); + try + { + if (!load_result.config) + return nullptr; + DictionaryStructure dictionary_structure = ExternalDictionariesLoader::getDictionaryStructure(*load_result.config); + return StorageDictionary::create(StorageID(database_name, load_result.name), load_result.name, dictionary_structure); + } + catch (Exception & e) + { + throw Exception( + fmt::format("Error while loading dictionary '{}.{}': {}", + database_name, load_result.name, e.displayText()), + e.code()); + } } }