diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c69d822e383..7eacb6434cc 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -781,6 +782,7 @@ try registerRemoteFileMetadatas(); registerSchedulerNodes(); registerResourceManagers(); + QueryPlanStepRegistry::registerPlanSteps(); CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 9754897d54d..909a8fe8244 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -67,12 +67,10 @@ namespace class QueryTreeBuilder { public: - explicit QueryTreeBuilder(ASTPtr query_, ContextPtr context_); - QueryTreeNodePtr getQueryTreeNode() - { - return query_tree_node; - } + QueryTreeNodePtr buildQueryTreeNode(ASTPtr query_, ContextPtr context_); + + std::shared_ptr buildTableFunction(const ASTPtr & table_function, const ContextPtr & context) const; private: QueryTreeNodePtr buildSelectOrUnionExpression(const ASTPtr & select_or_union_query, @@ -111,13 +109,15 @@ private: ColumnTransformersNodes buildColumnTransformers(const ASTPtr & matcher_expression, const ContextPtr & context) const; - ASTPtr query; - QueryTreeNodePtr query_tree_node; + // ASTPtr query; + // QueryTreeNodePtr query_tree_node; }; -QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_, ContextPtr context_) - : query(query_->clone()) +QueryTreeNodePtr QueryTreeBuilder::buildQueryTreeNode(ASTPtr query_, ContextPtr context_) { + auto query = query_->clone(); + QueryTreeNodePtr query_tree_node; + if (query->as() || query->as() || query->as()) @@ -126,6 +126,8 @@ QueryTreeBuilder::QueryTreeBuilder(ASTPtr query_, ContextPtr context_) query_tree_node = buildExpressionList(query, context_); else query_tree_node = buildExpression(query, context_); + + return query_tree_node; } QueryTreeNodePtr QueryTreeBuilder::buildSelectOrUnionExpression(const ASTPtr & select_or_union_query, @@ -772,6 +774,33 @@ QueryTreeNodePtr QueryTreeBuilder::buildWindow(const ASTPtr & window_definition, return window_node; } +std::shared_ptr QueryTreeBuilder::buildTableFunction(const ASTPtr & table_function, const ContextPtr & context) const +{ + auto & table_function_expression = table_function->as(); + + auto node = std::make_shared(table_function_expression.name); + + if (table_function_expression.arguments) + { + const auto & function_arguments_list = table_function_expression.arguments->as().children; + for (const auto & argument : function_arguments_list) + { + if (!node->getSettingsChanges().empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function '{}' has arguments after SETTINGS", + table_function_expression.formatForErrorMessage()); + + if (argument->as() || argument->as() || argument->as()) + node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, false /*is_subquery*/, {} /*cte_name*/, context)); + else if (const auto * ast_set = argument->as()) + node->setSettingsChanges(ast_set->changes); + else + node->getArguments().getNodes().push_back(buildExpression(argument, context)); + } + } + + return node; +} + QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select_query, const ContextPtr & context) const { if (!tables_in_select_query) @@ -854,31 +883,11 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select } else if (table_expression.table_function) { - auto & table_function_expression = table_expression.table_function->as(); - - auto node = std::make_shared(table_function_expression.name); - - if (table_function_expression.arguments) - { - const auto & function_arguments_list = table_function_expression.arguments->as().children; - for (const auto & argument : function_arguments_list) - { - if (!node->getSettingsChanges().empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function '{}' has arguments after SETTINGS", - table_function_expression.formatForErrorMessage()); - - if (argument->as() || argument->as() || argument->as()) - node->getArguments().getNodes().push_back(buildSelectOrUnionExpression(argument, false /*is_subquery*/, {} /*cte_name*/, context)); - else if (const auto * ast_set = argument->as()) - node->setSettingsChanges(ast_set->changes); - else - node->getArguments().getNodes().push_back(buildExpression(argument, context)); - } - } + auto node = buildTableFunction(table_expression.table_function, context); if (table_expression_modifiers) node->setTableExpressionModifiers(*table_expression_modifiers); - node->setAlias(table_function_expression.tryGetAlias()); + node->setAlias(table_expression.table_function->tryGetAlias()); node->setOriginalAST(table_expression.table_function); table_expressions.push_back(std::move(node)); @@ -1052,8 +1061,14 @@ ColumnTransformersNodes QueryTreeBuilder::buildColumnTransformers(const ASTPtr & QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context) { - QueryTreeBuilder builder(std::move(query), context); - return builder.getQueryTreeNode(); + QueryTreeBuilder builder; + return builder.buildQueryTreeNode(std::move(query), context); +} + +std::shared_ptr buildTableFunctionQueryTree(ASTPtr query, ContextPtr context) +{ + QueryTreeBuilder builder; + return builder.buildTableFunction(query, context); } } diff --git a/src/Analyzer/QueryTreeBuilder.h b/src/Analyzer/QueryTreeBuilder.h index acff62e07c9..e1ec8e0f2b2 100644 --- a/src/Analyzer/QueryTreeBuilder.h +++ b/src/Analyzer/QueryTreeBuilder.h @@ -18,4 +18,7 @@ namespace DB */ QueryTreeNodePtr buildQueryTree(ASTPtr query, ContextPtr context); +/// For table function only +class TableFunctionNode; +std::shared_ptr buildTableFunctionQueryTree(ASTPtr query, ContextPtr context); } diff --git a/src/Analyzer/Resolve/IdentifierResolver.cpp b/src/Analyzer/Resolve/IdentifierResolver.cpp index 80e7d1e4445..8ff0ce3a83d 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.cpp +++ b/src/Analyzer/Resolve/IdentifierResolver.cpp @@ -385,7 +385,7 @@ QueryTreeNodePtr IdentifierResolver::wrapExpressionNodeInTupleElement(QueryTreeN /// Resolve identifier functions implementation /// Try resolve table identifier from database catalog -QueryTreeNodePtr IdentifierResolver::tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier, ContextPtr context) +std::shared_ptr IdentifierResolver::tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier, ContextPtr context) { size_t parts_size = table_identifier.getPartsSize(); if (parts_size < 1 || parts_size > 2) diff --git a/src/Analyzer/Resolve/IdentifierResolver.h b/src/Analyzer/Resolve/IdentifierResolver.h index cdbd7610b5e..7402c69db72 100644 --- a/src/Analyzer/Resolve/IdentifierResolver.h +++ b/src/Analyzer/Resolve/IdentifierResolver.h @@ -21,6 +21,7 @@ class QueryExpressionsAliasVisitor ; class QueryNode; class JoinNode; class ColumnNode; +class TableNode; using ProjectionName = String; using ProjectionNames = std::vector; @@ -86,7 +87,7 @@ public: /// Resolve identifier functions - static QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier, ContextPtr context); + static std::shared_ptr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier, ContextPtr context); QueryTreeNodePtr tryResolveIdentifierFromCompoundExpression(const Identifier & expression_identifier, size_t identifier_bind_size, diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index a526805fd4d..81f4dfca19a 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -3395,13 +3395,8 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - auto set = std::make_shared(size_limits_for_set, 0, settings.transform_null_in); - - set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(result_block.getColumnsWithTypeAndName()); - set->finishInsert(); - - auto future_set = std::make_shared(std::move(set)); + auto hash = function_arguments[1]->getTreeHash(); + auto future_set = std::make_shared(hash, std::move(result_block), settings.transform_null_in, size_limits_for_set); /// Create constant set column for constant folding diff --git a/src/Analyzer/TableExpressionModifiers.h b/src/Analyzer/TableExpressionModifiers.h index 9b76c9bc0fd..4b3e2e76831 100644 --- a/src/Analyzer/TableExpressionModifiers.h +++ b/src/Analyzer/TableExpressionModifiers.h @@ -14,6 +14,7 @@ class TableExpressionModifiers public: using Rational = ASTSampleRatio::Rational; + TableExpressionModifiers() = default; TableExpressionModifiers(bool has_final_, std::optional sample_size_ratio_, std::optional sample_offset_ratio_) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 8a1c7d3988a..74d75436b93 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -31,6 +31,7 @@ #include #include #include +#include #include #include #include @@ -899,6 +900,12 @@ void Connection::sendQuery( } +void Connection::sendQueryPlan(const QueryPlan & query_plan) +{ + writeVarUInt(Protocol::Client::QueryPlan, *out); + query_plan.serialize(*out); +} + void Connection::sendCancel() { /// If we already disconnected. diff --git a/src/Client/Connection.h b/src/Client/Connection.h index e09d913f1ba..275d9391951 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -110,6 +110,8 @@ public: bool with_pending_data/* = false */, std::function process_progress_callback) override; + void sendQueryPlan(const QueryPlan & query_plan) override; + void sendCancel() override; void sendData(const Block & block, const String & name/* = "" */, bool scalar/* = false */) override; diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index dd8348ea04f..1aee60e395e 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -99,6 +99,23 @@ void HedgedConnections::sendScalarsData(Scalars & data) pipeline_for_new_replicas.add(send_scalars_data); } +void HedgedConnections::sendQueryPlan(const QueryPlan & query_plan) +{ + std::lock_guard lock(cancel_mutex); + + if (!sent_query) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send query plan: query not yet sent."); + + auto send_query_plan = [&query_plan](ReplicaState & replica) { replica.connection->sendQueryPlan(query_plan); }; + + for (auto & offset_state : offset_states) + for (auto & replica : offset_state.replicas) + if (replica.connection) + send_query_plan(replica); + + pipeline_for_new_replicas.add(send_query_plan); +} + void HedgedConnections::sendExternalTablesData(std::vector & data) { std::lock_guard lock(cancel_mutex); diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 7f538804e5a..e2b4f195ff4 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -92,6 +92,8 @@ public: ClientInfo & client_info, bool with_pending_data) override; + void sendQueryPlan(const QueryPlan & query_plan) override; + void sendReadTaskResponse(const String &) override { throw Exception(ErrorCodes::LOGICAL_ERROR, "sendReadTaskResponse in not supported with HedgedConnections"); diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h index 09211de53b0..cd4fe33ef2d 100644 --- a/src/Client/IConnections.h +++ b/src/Client/IConnections.h @@ -25,6 +25,8 @@ public: ClientInfo & client_info, bool with_pending_data) = 0; + virtual void sendQueryPlan(const QueryPlan & query_plan) = 0; + virtual void sendReadTaskResponse(const String &) = 0; virtual void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) = 0; diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 6ab4234bca2..cfc47c5d2e6 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -62,6 +62,7 @@ struct ExternalTableData using ExternalTableDataPtr = std::unique_ptr; using ExternalTablesData = std::vector; +class QueryPlan; class IServerConnection : boost::noncopyable { @@ -104,6 +105,8 @@ public: bool with_pending_data, std::function process_progress_callback) = 0; + virtual void sendQueryPlan(const QueryPlan & query_plan) = 0; + virtual void sendCancel() = 0; /// Send block of data; if name is specified, server will write it to external (temporary) table of that name. diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 7595a29912b..efa1eaac4b2 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -208,7 +208,7 @@ void LocalConnection::sendQuery( try { - state->io = executeQuery(state->query, query_context, QueryFlags{}, state->stage).second; + state->io = executeQuery(state->query, nullptr, query_context, QueryFlags{}, state->stage).second; if (state->io.pipeline.pushing()) { @@ -282,6 +282,11 @@ void LocalConnection::sendQuery( } } +void LocalConnection::sendQueryPlan(const QueryPlan &) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented"); +} + void LocalConnection::sendData(const Block & block, const String &, bool) { if (!block) diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index b424c5b5aa3..0ec34552d5f 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -116,6 +116,8 @@ public: bool with_pending_data/* = false */, std::function process_progress_callback) override; + void sendQueryPlan(const QueryPlan &) override; + void sendCancel() override; void sendData(const Block & block, const String & name/* = "" */, bool scalar/* = false */) override; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 244eccf1ed9..ffe77190d70 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -90,6 +90,21 @@ void MultiplexedConnections::sendScalarsData(Scalars & data) } } +void MultiplexedConnections::sendQueryPlan(const QueryPlan & query_plan) +{ + std::lock_guard lock(cancel_mutex); + + if (!sent_query) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot send scalars data: query not yet sent."); + + for (ReplicaState & state : replica_states) + { + Connection * connection = state.connection; + if (connection != nullptr) + connection->sendQueryPlan(query_plan); + } +} + void MultiplexedConnections::sendExternalTablesData(std::vector & data) { std::lock_guard lock(cancel_mutex); @@ -184,6 +199,7 @@ void MultiplexedConnections::sendQuery( sent_query = true; } + void MultiplexedConnections::sendIgnoredPartUUIDs(const std::vector & uuids) { std::lock_guard lock(cancel_mutex); diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index dec32e52d4f..1e9f0c96440 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -38,6 +38,8 @@ public: ClientInfo & client_info, bool with_pending_data) override; + void sendQueryPlan(const QueryPlan & query_plan) override; + void sendReadTaskResponse(const String &) override; void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) override; diff --git a/src/Columns/ColumnSet.h b/src/Columns/ColumnSet.h index 935a72e551a..95d2ab41eea 100644 --- a/src/Columns/ColumnSet.h +++ b/src/Columns/ColumnSet.h @@ -30,6 +30,7 @@ public: MutableColumnPtr cloneDummy(size_t s_) const override { return ColumnSet::create(s_, data); } FutureSetPtr getData() const { return data; } + void setData(FutureSetPtr data_) { data = std::move(data_); } // Used only for debugging, making it DUMPABLE Field operator[](size_t) const override { return {}; } diff --git a/src/Core/BaseSettings.h b/src/Core/BaseSettings.h index 6242d78aee7..311a8c55a09 100644 --- a/src/Core/BaseSettings.h +++ b/src/Core/BaseSettings.h @@ -99,6 +99,9 @@ public: void write(WriteBuffer & out, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT) const; void read(ReadBuffer & in, SettingsWriteFormat format = SettingsWriteFormat::DEFAULT); + void writeChangedBinary(WriteBuffer & out) const; + void readBinary(ReadBuffer & in); + // A debugging aid. std::string toString() const; @@ -499,6 +502,46 @@ void BaseSettings::write(WriteBuffer & out, SettingsWriteFormat format) BaseSettingsHelpers::writeString(std::string_view{}, out); } +template +void BaseSettings::writeChangedBinary(WriteBuffer & out) const +{ + const auto & accessor = Traits::Accessor::instance(); + + size_t num_settings = 0; + for (auto it = this->begin(); it != this->end(); ++it) + ++num_settings; + + writeVarUInt(num_settings, out); + + for (const auto & field : *this) + { + BaseSettingsHelpers::writeString(field.getName(), out); + using Flags = BaseSettingsHelpers::Flags; + Flags flags{0}; + BaseSettingsHelpers::writeFlags(flags, out); + accessor.writeBinary(*this, field.index, out); + } +} + +template +void BaseSettings::readBinary(ReadBuffer & in) +{ + const auto & accessor = Traits::Accessor::instance(); + + size_t num_settings = 0; + readVarUInt(num_settings, in); + + for (size_t i = 0; i < num_settings; ++i) + { + String read_name = BaseSettingsHelpers::readString(in); + std::string_view name = TTraits::resolveName(read_name); + size_t index = accessor.find(name); + + std::ignore = BaseSettingsHelpers::readFlags(in); + accessor.readBinary(*this, index, in); + } +} + template void BaseSettings::read(ReadBuffer & in, SettingsWriteFormat format) { diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 0e5b1bac000..93dfdad00b9 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -186,11 +186,7 @@ void readBinary(Tuple & x, ReadBuffer & buf) readBinary(size, buf); for (size_t index = 0; index < size; ++index) - { - UInt8 type; - readBinary(type, buf); - x.push_back(getBinaryValue(type, buf)); - } + x.push_back(readFieldBinary(buf)); } void writeBinary(const Tuple & x, WriteBuffer & buf) @@ -199,11 +195,7 @@ void writeBinary(const Tuple & x, WriteBuffer & buf) writeBinary(size, buf); for (const auto & elem : x) - { - const UInt8 type = elem.getType(); - writeBinary(type, buf); - Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem); - } + writeFieldBinary(elem, buf); } void writeText(const Tuple & x, WriteBuffer & buf) @@ -217,11 +209,7 @@ void readBinary(Map & x, ReadBuffer & buf) readBinary(size, buf); for (size_t index = 0; index < size; ++index) - { - UInt8 type; - readBinary(type, buf); - x.push_back(getBinaryValue(type, buf)); - } + x.push_back(readFieldBinary(buf)); } void writeBinary(const Map & x, WriteBuffer & buf) @@ -230,11 +218,7 @@ void writeBinary(const Map & x, WriteBuffer & buf) writeBinary(size, buf); for (const auto & elem : x) - { - const UInt8 type = elem.getType(); - writeBinary(type, buf); - Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem); - } + writeFieldBinary(elem, buf); } void writeText(const Map & x, WriteBuffer & buf) @@ -319,6 +303,20 @@ void writeFieldText(const Field & x, WriteBuffer & buf) buf.write(res.data(), res.size()); } +void writeFieldBinary(const Field & x, WriteBuffer & buf) +{ + const UInt8 type = x.getType(); + writeBinary(type, buf); + Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, x); +} + +Field readFieldBinary(ReadBuffer & buf) +{ + UInt8 type; + readBinary(type, buf); + return getBinaryValue(type, buf); +} + String Field::dump() const { diff --git a/src/Core/Field.h b/src/Core/Field.h index ba8c66580ad..28d778cdbea 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -1000,6 +1000,10 @@ void readQuoted(DecimalField & x, ReadBuffer & buf); void writeFieldText(const Field & x, WriteBuffer & buf); + +void writeFieldBinary(const Field & x, WriteBuffer & buf); +Field readFieldBinary(ReadBuffer & buf); + String toString(const Field & x); std::string_view fieldTypeToString(Field::Types::Which type); diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index 2e5b91e9b1b..e197db70f04 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -166,7 +166,10 @@ namespace Protocol SSHChallengeRequest = 11, /// Request SSH signature challenge SSHChallengeResponse = 12, /// Reply to SSH signature challenge - MAX = SSHChallengeResponse, + + QueryPlan = 13, /// Query plan + + MAX = QueryPlan, }; inline const char * toString(UInt64 packet) diff --git a/src/Core/QueryProcessingStage.h b/src/Core/QueryProcessingStage.h index ce6949c0dea..8b3d4a908a6 100644 --- a/src/Core/QueryProcessingStage.h +++ b/src/Core/QueryProcessingStage.h @@ -35,10 +35,15 @@ namespace QueryProcessingStage WithMergeableStateAfterAggregationAndLimit = 4, MAX = 5, + + /// QueryPlan is used + QueryPlan = 7, }; inline const char * toString(UInt64 stage) { + if (stage == QueryPlan) + return "QueryPlan"; static const char * data[] = { "FetchColumns", diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fadc4079fe0..08454105d90 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -928,6 +928,7 @@ class IColumn; M(Bool, allow_experimental_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\ + M(Bool, serialize_query_plan, true, "Serialize query plan for distributed processing", 0)\ \ M(Bool, allow_statistics_optimize, false, "Allows using statistics to optimize queries", 0) ALIAS(allow_statistic_optimize) \ M(Bool, allow_experimental_statistics, false, "Allows using statistics", 0) ALIAS(allow_experimental_statistic) \ diff --git a/src/Core/SortDescription.cpp b/src/Core/SortDescription.cpp index 1b3f81f8547..01f22b9b9e5 100644 --- a/src/Core/SortDescription.cpp +++ b/src/Core/SortDescription.cpp @@ -15,6 +15,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + void dumpSortDescription(const SortDescription & description, WriteBuffer & out) { bool first = true; @@ -209,4 +214,58 @@ JSONBuilder::ItemPtr explainSortDescription(const SortDescription & description) return json_array; } +void serializeSortDescription(const SortDescription & sort_description, WriteBuffer & out) +{ + writeVarUInt(sort_description.size(), out); + for (const auto & desc : sort_description) + { + writeStringBinary(desc.column_name, out); + + UInt8 flags = 0; + if (desc.direction > 0) + flags |= 1; + if (desc.nulls_direction > 0) + flags |= 2; + if (desc.collator) + flags |= 4; + if (desc.with_fill) + flags |= 8; + + writeIntBinary(flags, out); + + if (desc.collator) + writeStringBinary(desc.collator->getLocale(), out); + + if (desc.with_fill) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH FILL is not supported in serialized sort description"); + } +} + +void deserializeSortDescription(SortDescription & sort_description, ReadBuffer & in) +{ + size_t size = 0; + readVarUInt(size, in); + sort_description.resize(size); + for (auto & desc : sort_description) + { + readStringBinary(desc.column_name, in); + UInt8 flags = 0; + readIntBinary(flags, in); + + desc.direction = (flags & 1) ? 1 : -1; + desc.nulls_direction = (flags & 2) ? 1 : -1; + + if (flags & 4) + { + String collator_locale; + readStringBinary(collator_locale, in); + if (!collator_locale.empty()) + desc.collator = std::make_shared(collator_locale); + } + + if (flags & 8) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "WITH FILL is not supported in deserialized sort description"); + } +} + } diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index 5c6f3e3150a..a87872193eb 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -140,4 +140,11 @@ void dumpSortDescription(const SortDescription & description, WriteBuffer & out) std::string dumpSortDescription(const SortDescription & description); JSONBuilder::ItemPtr explainSortDescription(const SortDescription & description); + +class WriteBuffer; +class ReadBuffer; + +void serializeSortDescription(const SortDescription & sort_description, WriteBuffer & out); +void deserializeSortDescription(SortDescription & sort_description, ReadBuffer & in); + } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 1bc6334db29..be4e678a4b0 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1213,14 +1213,14 @@ void DatabaseReplicated::recoverLostReplica(const ZooKeeperPtr & current_zookeep String query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Ordinary", backQuoteIfNeed(to_db_name)); auto query_context = Context::createCopy(getContext()); query_context->setSetting("allow_deprecated_database_ordinary", 1); - executeQuery(query, query_context, QueryFlags{ .internal = true }); + executeQuery(query, nullptr, query_context, QueryFlags{ .internal = true }); /// But we want to avoid discarding UUID of ReplicatedMergeTree tables, because it will not work /// if zookeeper_path contains {uuid} macro. Replicated database do not recreate replicated tables on recovery, /// so it's ok to save UUID of replicated table. query = fmt::format("CREATE DATABASE IF NOT EXISTS {} ENGINE=Atomic", backQuoteIfNeed(to_db_name_replicated)); query_context = Context::createCopy(getContext()); - executeQuery(query, query_context, QueryFlags{ .internal = true }); + executeQuery(query, nullptr, query_context, QueryFlags{ .internal = true }); } size_t moved_tables = 0; diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 1364e9ae2b2..af54b420719 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -112,7 +112,7 @@ static BlockIO tryToExecuteQuery(const String & query_to_execute, ContextMutable if (!database.empty()) query_context->setCurrentDatabase(database); - return executeQuery("/*" + comment + "*/ " + query_to_execute, query_context, QueryFlags{ .internal = true }).second; + return executeQuery("/*" + comment + "*/ " + query_to_execute, nullptr, query_context, QueryFlags{ .internal = true }).second; } catch (...) { diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index b36d53a6159..80afef393f9 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -170,7 +170,7 @@ QueryPipeline ClickHouseDictionarySource::createStreamForQuery(const String & qu if (configuration.is_local) { - pipeline = executeQuery(query, context_copy, QueryFlags{ .internal = true }).second.pipeline; + pipeline = executeQuery(query, nullptr, context_copy, QueryFlags{ .internal = true }).second.pipeline; pipeline.convertStructureTo(empty_sample_block.getColumnsWithTypeAndName()); } else @@ -192,7 +192,7 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re if (configuration.is_local) { - return readInvalidateQuery(executeQuery(request, context_copy, QueryFlags{ .internal = true }).second.pipeline); + return readInvalidateQuery(executeQuery(request, nullptr, context_copy, QueryFlags{ .internal = true }).second.pipeline); } else { diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index fb5109eaa88..e69b9a1a244 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -35,8 +35,14 @@ public: String getName() const override { return "FunctionExpression"; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { + if (input_rows_count == 0) + return result_type->createColumn(); + + if (!expression_actions) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No actions were passed to FunctionExpression"); + DB::Block expr_columns; for (size_t i = 0; i < arguments.size(); ++i) { @@ -116,8 +122,8 @@ public: using CapturePtr = std::shared_ptr; - ExecutableFunctionCapture(ExpressionActionsPtr expression_actions_, CapturePtr capture_) - : expression_actions(std::move(expression_actions_)), capture(std::move(capture_)) {} + ExecutableFunctionCapture(std::shared_ptr actions_dag_, CapturePtr capture_) + : actions_dag(std::move(actions_dag_)), capture(std::move(capture_)) {} String getName() const override { return "FunctionCapture"; } @@ -151,9 +157,18 @@ public: return ColumnFunction::create(input_rows_count, std::move(function), arguments); } + void buildExpressionActions(const ExpressionActionsSettings & settings) + { + /// In rare situations this function can be called a few times (e.g. ActonsDAG was clonned). + if (!expression_actions) + expression_actions = std::make_shared(actions_dag->clone(), settings); + } + private: - ExpressionActionsPtr expression_actions; + std::shared_ptr actions_dag; CapturePtr capture; + + ExpressionActionsPtr expression_actions; }; class FunctionCapture : public IFunctionBase @@ -162,11 +177,11 @@ public: using CapturePtr = ExecutableFunctionCapture::CapturePtr; FunctionCapture( - ExpressionActionsPtr expression_actions_, + std::shared_ptr actions_dag_, CapturePtr capture_, DataTypePtr return_type_, String name_) - : expression_actions(std::move(expression_actions_)) + : actions_dag(std::move(actions_dag_)) , capture(std::move(capture_)) , return_type(std::move(return_type_)) , name(std::move(name_)) @@ -182,11 +197,14 @@ public: ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override { - return std::make_unique(expression_actions, capture); + return std::make_unique(actions_dag, capture); } + const ExecutableFunctionCapture::Capture & getCapture() const { return *capture; } + const ActionsDAG & getAcionsDAG() const { return *actions_dag; } + private: - ExpressionActionsPtr expression_actions; + std::shared_ptr actions_dag; CapturePtr capture; DataTypePtr return_type; String name; @@ -199,23 +217,21 @@ public: using CapturePtr = ExecutableFunctionCapture::CapturePtr; FunctionCaptureOverloadResolver( - ExpressionActionsPtr expression_actions_, + std::shared_ptr actions_dag_, const Names & captured_names, const NamesAndTypesList & lambda_arguments, const DataTypePtr & function_return_type, const String & expression_return_name) - : expression_actions(std::move(expression_actions_)) + : actions_dag(std::move(actions_dag_)) { /// Check that expression does not contain unusual actions that will break columns structure. - for (const auto & action : expression_actions->getActions()) - if (action.node->type == ActionsDAG::ActionType::ARRAY_JOIN) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression with arrayJoin or other unusual action cannot be captured"); + if (actions_dag->hasArrayJoin()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression with arrayJoin or other unusual action cannot be captured"); std::unordered_map arguments_map; - const auto & all_arguments = expression_actions->getRequiredColumnsWithTypes(); - for (const auto & arg : all_arguments) - arguments_map[arg.name] = arg.type; + for (const auto * input : actions_dag->getInputs()) + arguments_map[input->result_name] = input->result_type; DataTypes captured_types; captured_types.reserve(captured_names.size()); @@ -259,11 +275,11 @@ public: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName &, const DataTypePtr &) const override { - return std::make_unique(expression_actions, capture, return_type, name); + return std::make_unique(actions_dag, capture, return_type, name); } private: - ExpressionActionsPtr expression_actions; + std::shared_ptr actions_dag; CapturePtr capture; DataTypePtr return_type; String name; diff --git a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp index 71fc1047cfa..88b28fa4153 100644 --- a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp @@ -24,7 +24,7 @@ InterpreterShowAccessEntitiesQuery::InterpreterShowAccessEntitiesQuery(const AST BlockIO InterpreterShowAccessEntitiesQuery::execute() { - return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; + return executeQuery(getRewrittenQuery(), nullptr, getContext(), QueryFlags{ .internal = true }).second; } diff --git a/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp index 42b7fc51022..1773c56c22b 100644 --- a/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowPrivilegesQuery.cpp @@ -13,7 +13,7 @@ InterpreterShowPrivilegesQuery::InterpreterShowPrivilegesQuery(const ASTPtr & qu BlockIO InterpreterShowPrivilegesQuery::execute() { - return executeQuery("SELECT * FROM system.privileges", context, QueryFlags{ .internal = true }).second; + return executeQuery("SELECT * FROM system.privileges", nullptr, context, QueryFlags{ .internal = true }).second; } void registerInterpreterShowPrivilegesQuery(InterpreterFactory & factory) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 44862344df0..b92f8bd7c9e 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -3,15 +3,20 @@ #include #include #include +#include +#include #include +#include #include #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -38,6 +43,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int BAD_ARGUMENTS; + extern const int INCORRECT_DATA; } namespace @@ -3218,4 +3224,347 @@ const ActionsDAG::Node * FindAliasForInputName::find(const String & name) return it->second; } +static const ColumnSet * tryGetColumnSet(const ColumnPtr & colunm) +{ + const IColumn * maybe_set = colunm.get(); + if (const auto * column_const = typeid_cast(maybe_set)) + maybe_set = &column_const->getDataColumn(); + + return typeid_cast(maybe_set); +} + +static void serializeCapture(const ExecutableFunctionCapture::Capture & capture, WriteBuffer & out) +{ + writeStringBinary(capture.return_name, out); + encodeDataType(capture.return_type, out); + + writeVarUInt(capture.captured_names.size(), out); + for (const auto & name : capture.captured_names) + writeStringBinary(name, out); + + writeVarUInt(capture.captured_types.size(), out); + for (const auto & type : capture.captured_types) + encodeDataType(type, out); + + writeVarUInt(capture.lambda_arguments.size(), out); + for (const auto & item : capture.lambda_arguments) + { + writeStringBinary(item.name, out); + encodeDataType(item.type, out); + } +} + +static void deserializeCapture(ExecutableFunctionCapture::Capture & capture, ReadBuffer & in) +{ + readStringBinary(capture.return_name, in); + capture.return_type = decodeDataType(in); + + UInt64 num_names; + readVarUInt(num_names, in); + capture.captured_names.resize(num_names); + for (auto & name : capture.captured_names) + readStringBinary(name, in); + + UInt64 num_types; + readVarUInt(num_types, in); + capture.captured_types.resize(num_types); + for (auto & type : capture.captured_types) + type = decodeDataType(in); + + UInt64 num_args; + readVarUInt(num_args, in); + capture.lambda_arguments.clear(); + for (size_t i = 0; i < num_args; ++i) + { + NameAndTypePair name_and_type; + readStringBinary(name_and_type.name, in); + name_and_type.type = decodeDataType(in); + capture.lambda_arguments.push_back(std::move(name_and_type)); + } +} + +void ActionsDAG::serialize(WriteBuffer & out, SerializedSetsRegistry & registry) const +{ + size_t nodes_size = nodes.size(); + writeVarUInt(nodes_size, out); + + std::unordered_map node_to_id; + for (const auto & node : nodes) + node_to_id.emplace(&node, node_to_id.size()); + + if (nodes_size != node_to_id.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate nodes in ActionsDAG"); + + for (const auto & node : nodes) + { + writeIntBinary(static_cast(node.type), out); + writeStringBinary(node.result_name, out); + encodeDataType(node.result_type, out); + + writeVarUInt(node.children.size(), out); + for (const auto * child : node.children) + writeVarUInt(node_to_id.at(child), out); + + /// Serialize column if it is present + const bool has_column = node.column != nullptr; + const ColumnSet * column_set = nullptr; + UInt8 column_flags = 0; + if (has_column) + { + column_flags |= 1; + if (node.is_deterministic_constant) + column_flags |= 2; + + column_set = tryGetColumnSet(node.column); + if (column_set) + column_flags |= 4; + } + + const auto * function_capture = typeid_cast(node.function_base.get()); + if (function_capture) + column_flags |= 8; + + writeIntBinary(column_flags, out); + if (column_set) + { + auto hash = column_set->getData()->getHash(); + writeBinary(hash, out); + registry.sets.emplace(hash, column_set->getData()); + } + else if (has_column) + { + const auto * const_column = typeid_cast(node.column.get()); + if (!const_column) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot serialize non-constant column {}", node.column->getName()); + + auto value = const_column->getField(); + node.result_type->getDefaultSerialization()->serializeBinary(value, out, FormatSettings{}); + } + + if (node.type == ActionType::INPUT) + { + /// nothing to serialize + } + else if (node.type == ActionType::COLUMN) + { + /// nothing to serialize, column is already serialized + } + else if (node.type == ActionType::ALIAS) + { + /// nothing to serialize + } + else if (node.type == ActionType::FUNCTION) + { + writeStringBinary(node.function_base->getName(), out); + if (function_capture) + { + serializeCapture(function_capture->getCapture(), out); + function_capture->getAcionsDAG().serialize(out, registry); + } + } + else if (node.type == ActionType::ARRAY_JOIN) + { + /// nothing to serialize + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown node type {}", static_cast(node.type)); + } + } + + writeVarUInt(inputs.size(), out); + for (const auto * input : inputs) + writeVarUInt(node_to_id.at(input), out); + + writeVarUInt(outputs.size(), out); + for (const auto * output : outputs) + writeVarUInt(node_to_id.at(output), out); +} + +ActionsDAG ActionsDAG::deserialize(ReadBuffer & in, DeserializedSetsRegistry & registry, const ContextPtr & context) +{ + size_t nodes_size; + readVarUInt(nodes_size, in); + + std::list nodes; + std::unordered_map id_to_node; + for (size_t i = 0; i < nodes_size; ++i) + id_to_node.emplace(i, &nodes.emplace_back(Node{})); + + for (size_t i = 0; i < nodes_size; ++i) + { + Node & node = *id_to_node.at(i); + + UInt8 action_type{0}; + readIntBinary(action_type, in); + if (action_type > static_cast(ActionType::FUNCTION)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown action type {}", size_t(action_type)); + node.type = static_cast(action_type); + + readStringBinary(node.result_name, in); + node.result_type = decodeDataType(in); + + size_t children_size; + readVarUInt(children_size, in); + for (size_t j = 0; j < children_size; ++j) + { + size_t child_id; + readVarUInt(child_id, in); + node.children.push_back(id_to_node.at(child_id)); + } + + UInt8 column_flags = 0; + readIntBinary(column_flags, in); + + /// Deserialize column if it is present + if (column_flags & 1) + { + if ((column_flags & 2) == 0) + node.is_deterministic_constant = false; + + if (column_flags & 4) + { + FutureSet::Hash hash; + readBinary(hash, in); + + auto column_set = ColumnSet::create(0, nullptr); + registry.sets[hash].push_back(column_set.get()); + + node.column = std::move(column_set); + } + else + { + Field value; + node.result_type->getDefaultSerialization()->deserializeBinary(value, in, FormatSettings{}); + node.column = node.result_type->createColumnConst(0, value); + } + } + + if (node.type == ActionType::INPUT) + { + /// nothing to deserialize + if (!node.children.empty()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Deserialized input can't have children"); + } + else if (node.type == ActionType::COLUMN) + { + /// Column is already deserialized + if (!node.children.empty()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Deserialized column can't have children"); + } + else if (node.type == ActionType::ALIAS) + { + /// nothing to deserialize + if (node.children.size() != 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "Deserialized alias must have one children"); + } + else if (node.type == ActionType::FUNCTION) + { + String function_name; + readStringBinary(function_name, in); + + ColumnsWithTypeAndName arguments; + arguments.reserve(node.children.size()); + for (const auto * child : node.children) + { + ColumnWithTypeAndName argument; + argument.column = child->column; + argument.type = child->result_type; + argument.name = child->result_name; + + arguments.emplace_back(std::move(argument)); + } + + if (column_flags & 8) + { + ExecutableFunctionCapture::Capture capture; + deserializeCapture(capture, in); + auto capture_dag = ActionsDAG::deserialize(in, registry, context); + + node.function_base = std::make_shared( + std::make_shared(std::move(capture_dag)), + std::make_shared(std::move(capture)), + node.result_type, + function_name); + + node.function = node.function_base->prepare(arguments); + node.is_function_compiled = false; + } + else + { + auto function = FunctionFactory::instance().get(function_name, context); + + node.function_base = function->build(arguments); + node.function = node.function_base->prepare(arguments); + node.is_function_compiled = false; + + if (!node.function_base->getResultType()->equals(*node.result_type)) + throw Exception(ErrorCodes::INCORRECT_DATA, + "Deserialized function {} has type. Expected {}, deserialized {}.", + function_name, + node.function_base->getResultType()->getName(), + node.result_type->getName()); + } + } + else if (node.type == ActionType::ARRAY_JOIN) + { + /// nothing to deserialize + if (node.children.size() != 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "Deserialized array join must have one children"); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown node type {}", static_cast(node.type)); + } + } + + size_t inputs_size; + readVarUInt(inputs_size, in); + std::vector inputs; + std::unordered_set inputs_set; + inputs.reserve(inputs_size); + for (size_t i = 0; i < inputs_size; ++i) + { + size_t input_id; + readVarUInt(input_id, in); + const auto * input = id_to_node.at(input_id); + + if (input->type != ActionType::INPUT) + throw Exception(ErrorCodes::INCORRECT_DATA, + "Deserialized input {} has type {}", + input->result_name, magic_enum::enum_name(input->type)); + + if (!inputs_set.emplace(input).second) + throw Exception(ErrorCodes::INCORRECT_DATA, + "Duplicate input {}", input->result_name); + + inputs.push_back(input); + } + + size_t outputs_size; + readVarUInt(outputs_size, in); + std::vector outputs; + outputs.reserve(outputs_size); + for (size_t i = 0; i < outputs_size; ++i) + { + size_t output_id; + readVarUInt(output_id, in); + outputs.push_back(id_to_node.at(output_id)); + } + + for (const auto & node : nodes) + if (node.type == ActionType::INPUT && !inputs_set.contains(&node)) + throw Exception(ErrorCodes::INCORRECT_DATA, + "Deserialized input {} is not in the inputs list", + node.result_name); + + ActionsDAG dag; + dag.nodes = std::move(nodes); + dag.inputs = std::move(inputs); + dag.outputs = std::move(outputs); + + return dag; +} + } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 0d6b1ce0e04..71aa618cbbf 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -35,6 +35,9 @@ namespace JSONBuilder class SortDescription; +struct SerializedSetsRegistry; +struct DeserializedSetsRegistry; + /// Directed acyclic graph of expressions. /// This is an intermediate representation of actions which is usually built from expression list AST. /// Node of DAG describe calculation of a single column with known type, name, and constant value (if applicable). @@ -130,6 +133,9 @@ public: std::string dumpNames() const; std::string dumpDAG() const; + void serialize(WriteBuffer & out, SerializedSetsRegistry & registry) const; + static ActionsDAG deserialize(ReadBuffer & in, DeserializedSetsRegistry & registry, const ContextPtr & context); + const Node & addInput(std::string name, DataTypePtr type); const Node & addInput(ColumnWithTypeAndName column); const Node & addColumn(ColumnWithTypeAndName column); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 368eb8174f0..60441624125 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1286,14 +1286,10 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & String result_name = lambda->arguments->children.at(1)->getColumnName(); lambda_dag.removeUnusedActions(Names(1, result_name)); - auto lambda_actions = std::make_shared( - std::move(lambda_dag), - ExpressionActionsSettings::fromContext(data.getContext(), CompileExpressions::yes)); - - DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; + DataTypePtr result_type = lambda_dag.findInOutputs(result_name).result_type; Names captured; - Names required = lambda_actions->getRequiredColumns(); + Names required = lambda_dag.getRequiredColumnsNames(); for (const auto & required_arg : required) if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end()) captured.push_back(required_arg); @@ -1303,7 +1299,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & String lambda_name = data.getUniqueName("__lambda"); auto function_capture = std::make_shared( - lambda_actions, captured, lambda_arguments, result_type, result_name); + std::make_shared(std::move(lambda_dag)), captured, lambda_arguments, result_type, result_name); data.addFunction(function_capture, captured, lambda_name); argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); @@ -1451,7 +1447,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return set; if (StorageSet * storage_set = dynamic_cast(table.get())) - return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); + return data.prepared_sets->addFromStorage(set_key, storage_set->getSet(), table_id); } if (!data.getContext()->isGlobalContext()) diff --git a/src/Interpreters/AggregateDescription.cpp b/src/Interpreters/AggregateDescription.cpp index d4c09995b56..a83c94234c6 100644 --- a/src/Interpreters/AggregateDescription.cpp +++ b/src/Interpreters/AggregateDescription.cpp @@ -1,8 +1,10 @@ #include +#include #include #include #include #include +#include namespace DB @@ -121,4 +123,63 @@ void AggregateDescription::explain(JSONBuilder::JSONMap & map) const map.add("Arguments", std::move(args_array)); } +void serializeAggregateDescriptions(const AggregateDescriptions & aggregates, WriteBuffer & out) +{ + writeVarUInt(aggregates.size(), out); + for (const auto & aggregate : aggregates) + { + writeStringBinary(aggregate.column_name, out); + + writeVarUInt(aggregate.argument_names.size(), out); + for (const auto & name : aggregate.argument_names) + writeStringBinary(name, out); + + writeStringBinary(aggregate.function->getName(), out); + + writeVarUInt(aggregate.parameters.size(), out); + for (const auto & param : aggregate.parameters) + writeFieldBinary(param, out); + } +} + +void deserializeAggregateDescriptions(AggregateDescriptions & aggregates, ReadBuffer & in, const Block & header) +{ + UInt64 num_aggregates; + readVarUInt(num_aggregates, in); + aggregates.resize(num_aggregates); + for (auto & aggregate : aggregates) + { + readStringBinary(aggregate.column_name, in); + + UInt64 num_args; + readVarUInt(num_args, in); + aggregate.argument_names.resize(num_args); + for (auto & arg_name : aggregate.argument_names) + readStringBinary(arg_name, in); + + String function_name; + readStringBinary(function_name, in); + + UInt64 num_params; + readVarUInt(num_params, in); + aggregate.parameters.resize(num_params); + for (auto & param : aggregate.parameters) + param = readFieldBinary(in); + + DataTypes argument_types; + argument_types.reserve(num_args); + for (const auto & arg_name : aggregate.argument_names) + { + const auto & arg = header.getByName(arg_name); + argument_types.emplace_back(arg.type); + } + + auto action = NullsAction::EMPTY; /// As I understand, it should be resolved to function name. + AggregateFunctionProperties properties; + aggregate.function = AggregateFunctionFactory::instance().get( + function_name, action, argument_types, aggregate.parameters, properties); + } + +} + } diff --git a/src/Interpreters/AggregateDescription.h b/src/Interpreters/AggregateDescription.h index 0f1c0ce67ae..c7f88c865c6 100644 --- a/src/Interpreters/AggregateDescription.h +++ b/src/Interpreters/AggregateDescription.h @@ -25,4 +25,10 @@ struct AggregateDescription }; using AggregateDescriptions = std::vector; + +class Block; + +void serializeAggregateDescriptions(const AggregateDescriptions & aggregates, WriteBuffer & out); +void deserializeAggregateDescriptions(AggregateDescriptions & aggregates, ReadBuffer & in, const Block & header); + } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 4de0a640219..d9686301026 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -139,7 +139,7 @@ public: bool optimize_group_by_constant_keys; - const double min_hit_rate_to_use_consecutive_keys_optimization; + const float min_hit_rate_to_use_consecutive_keys_optimization; StatsCollectingParams stats_collecting_params; @@ -162,7 +162,7 @@ public: bool enable_prefetch_, bool only_merge_, // true for projections bool optimize_group_by_constant_keys_, - double min_hit_rate_to_use_consecutive_keys_optimization_, + float min_hit_rate_to_use_consecutive_keys_optimization_, const StatsCollectingParams & stats_collecting_params_) : keys(keys_) , aggregates(aggregates_) @@ -190,7 +190,7 @@ public: } /// Only parameters that matter during merge. - Params(const Names & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_, size_t max_block_size_, double min_hit_rate_to_use_consecutive_keys_optimization_) + Params(const Names & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_, size_t max_block_size_, float min_hit_rate_to_use_consecutive_keys_optimization_) : Params( keys_, aggregates_, overflow_row_, 0, OverflowMode::THROW, 0, 0, 0, false, nullptr, max_threads_, 0, false, 0, max_block_size_, false, true, false, min_hit_rate_to_use_consecutive_keys_optimization_, {}) { diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index e35d31d2350..7eefdb03d40 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -165,14 +165,26 @@ void SelectStreamFactory::createForShardImpl( auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) { Block shard_header; - if (context->getSettingsRef().allow_experimental_analyzer) - shard_header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree, context, SelectQueryOptions(processed_stage).analyze()); + std::unique_ptr query_plan; + if (context->getSettingsRef().serialize_query_plan) + { + query_plan = createLocalPlan( + query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, has_missing_objects, true); + + shard_header = query_plan->getCurrentDataStream().header; + } else - shard_header = header; + { + if (context->getSettingsRef().allow_experimental_analyzer) + shard_header = InterpreterSelectQueryAnalyzer::getSampleBlock(query_tree, context, SelectQueryOptions(processed_stage).analyze()); + else + shard_header = header; + } remote_shards.emplace_back(Shard{ .query = query_ast, .query_tree = query_tree, + .query_plan = std::move(query_plan), .main_table = main_table, .header = shard_header, .has_missing_objects = has_missing_objects, diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 2e7b2445c6b..f15c23c2c3c 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -53,6 +53,8 @@ public: ASTPtr query; QueryTreeNodePtr query_tree; + std::shared_ptr query_plan; + /// Used to check the table existence on remote node StorageID main_table; Block header; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index edf419d404e..adde24bb5cd 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include @@ -54,6 +55,11 @@ ExpressionActions::ExpressionActions(ActionsDAG actions_dag_, const ExpressionAc , project_inputs(project_inputs_) , settings(settings_) { + for (const auto & node : actions_dag.getNodes()) + if (node.type == ActionsDAG::ActionType::FUNCTION) + if (auto * function_capture = typeid_cast(node.function.get())) + function_capture->buildExpressionActions(settings); + /// It's important to determine lazy executed nodes before compiling expressions. std::unordered_set lazy_executed_nodes = processShortCircuitFunctions(actions_dag, settings.short_circuit_function_evaluation); diff --git a/src/Interpreters/InterpreterKillQueryQuery.cpp b/src/Interpreters/InterpreterKillQueryQuery.cpp index 2c579f3b468..859c9872cfe 100644 --- a/src/Interpreters/InterpreterKillQueryQuery.cpp +++ b/src/Interpreters/InterpreterKillQueryQuery.cpp @@ -425,7 +425,7 @@ Block InterpreterKillQueryQuery::getSelectResult(const String & columns, const S if (where_expression) select_query += " WHERE " + queryToString(where_expression); - auto io = executeQuery(select_query, getContext(), QueryFlags{ .internal = true }).second; + auto io = executeQuery(select_query, nullptr, getContext(), QueryFlags{ .internal = true }).second; PullingPipelineExecutor executor(io.pipeline); Block res; while (!res && executor.pull(res)); diff --git a/src/Interpreters/InterpreterShowColumnsQuery.cpp b/src/Interpreters/InterpreterShowColumnsQuery.cpp index 472cdedf3ae..b8c620a14f7 100644 --- a/src/Interpreters/InterpreterShowColumnsQuery.cpp +++ b/src/Interpreters/InterpreterShowColumnsQuery.cpp @@ -164,7 +164,7 @@ WHERE BlockIO InterpreterShowColumnsQuery::execute() { - return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; + return executeQuery(getRewrittenQuery(), nullptr, getContext(), QueryFlags{ .internal = true }).second; } void registerInterpreterShowColumnsQuery(InterpreterFactory & factory) diff --git a/src/Interpreters/InterpreterShowEngineQuery.cpp b/src/Interpreters/InterpreterShowEngineQuery.cpp index f2d057a3fcf..8a2da6792d6 100644 --- a/src/Interpreters/InterpreterShowEngineQuery.cpp +++ b/src/Interpreters/InterpreterShowEngineQuery.cpp @@ -13,7 +13,7 @@ namespace DB BlockIO InterpreterShowEnginesQuery::execute() { - return executeQuery("SELECT * FROM system.table_engines ORDER BY name", getContext(), QueryFlags{ .internal = true }).second; + return executeQuery("SELECT * FROM system.table_engines ORDER BY name", nullptr, getContext(), QueryFlags{ .internal = true }).second; } void registerInterpreterShowEnginesQuery(InterpreterFactory & factory) diff --git a/src/Interpreters/InterpreterShowFunctionsQuery.cpp b/src/Interpreters/InterpreterShowFunctionsQuery.cpp index 96a730f6527..c088875f814 100644 --- a/src/Interpreters/InterpreterShowFunctionsQuery.cpp +++ b/src/Interpreters/InterpreterShowFunctionsQuery.cpp @@ -16,7 +16,7 @@ InterpreterShowFunctionsQuery::InterpreterShowFunctionsQuery(const ASTPtr & quer BlockIO InterpreterShowFunctionsQuery::execute() { - return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; + return executeQuery(getRewrittenQuery(), nullptr, getContext(), QueryFlags{ .internal = true }).second; } String InterpreterShowFunctionsQuery::getRewrittenQuery() diff --git a/src/Interpreters/InterpreterShowIndexesQuery.cpp b/src/Interpreters/InterpreterShowIndexesQuery.cpp index 31f0404e123..8e2a5ab93c2 100644 --- a/src/Interpreters/InterpreterShowIndexesQuery.cpp +++ b/src/Interpreters/InterpreterShowIndexesQuery.cpp @@ -123,7 +123,7 @@ ORDER BY index_type, expression, seq_in_index;)", database, table, where_express BlockIO InterpreterShowIndexesQuery::execute() { - return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; + return executeQuery(getRewrittenQuery(), nullptr, getContext(), QueryFlags{ .internal = true }).second; } void registerInterpreterShowIndexesQuery(InterpreterFactory & factory) diff --git a/src/Interpreters/InterpreterShowProcesslistQuery.cpp b/src/Interpreters/InterpreterShowProcesslistQuery.cpp index 7bdb94482da..9200c77b0c2 100644 --- a/src/Interpreters/InterpreterShowProcesslistQuery.cpp +++ b/src/Interpreters/InterpreterShowProcesslistQuery.cpp @@ -13,7 +13,7 @@ namespace DB BlockIO InterpreterShowProcesslistQuery::execute() { - return executeQuery("SELECT * FROM system.processes ORDER BY elapsed DESC", getContext(), QueryFlags{ .internal = true }).second; + return executeQuery("SELECT * FROM system.processes ORDER BY elapsed DESC", nullptr, getContext(), QueryFlags{ .internal = true }).second; } void registerInterpreterShowProcesslistQuery(InterpreterFactory & factory) diff --git a/src/Interpreters/InterpreterShowSettingQuery.cpp b/src/Interpreters/InterpreterShowSettingQuery.cpp index 90acaa7b083..2eba0e173f3 100644 --- a/src/Interpreters/InterpreterShowSettingQuery.cpp +++ b/src/Interpreters/InterpreterShowSettingQuery.cpp @@ -27,7 +27,7 @@ String InterpreterShowSettingQuery::getRewrittenQuery() BlockIO InterpreterShowSettingQuery::execute() { - return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; + return executeQuery(getRewrittenQuery(), nullptr, getContext(), QueryFlags{ .internal = true }).second; } void registerInterpreterShowSettingQuery(InterpreterFactory & factory) diff --git a/src/Interpreters/InterpreterShowTablesQuery.cpp b/src/Interpreters/InterpreterShowTablesQuery.cpp index 78580f2ec02..548940d8b16 100644 --- a/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -224,7 +224,7 @@ BlockIO InterpreterShowTablesQuery::execute() return res; } - return executeQuery(getRewrittenQuery(), getContext(), QueryFlags{ .internal = true }).second; + return executeQuery(getRewrittenQuery(), nullptr, getContext(), QueryFlags{ .internal = true }).second; } /// (*) Sorting is strictly speaking not necessary but 1. it is convenient for users, 2. SQL currently does not allow to diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h index 484fd6a0207..318dc4cc79e 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h @@ -75,7 +75,7 @@ public: ASTs rewritten_queries = InterpreterImpl::getRewrittenQueries(query, getContext(), mapped_to_database, mysql_database); for (const auto & rewritten_query : rewritten_queries) - executeQuery("/* Rewritten MySQL DDL Query */ " + queryToString(rewritten_query), getContext(), QueryFlags{ .internal = true }); + executeQuery("/* Rewritten MySQL DDL Query */ " + queryToString(rewritten_query), nullptr, getContext(), QueryFlags{ .internal = true }); return BlockIO{}; } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 43a04f8aff0..a4891adc012 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -26,7 +26,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static SizeLimits getSizeLimitsForSet(const Settings & settings) +SizeLimits PreparedSets::getSizeLimitsForSet(const Settings & settings) { return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); } @@ -47,8 +47,10 @@ static bool equals(const DataTypes & lhs, const DataTypes & rhs) } -FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} +FutureSetFromStorage::FutureSetFromStorage(Hash hash_, SetPtr set_, std::optional storage_id_) + : hash(hash_), storage_id(std::move(storage_id_)), set(std::move(set_)) {} SetPtr FutureSetFromStorage::get() const { return set; } +FutureSet::Hash FutureSetFromStorage::getHash() const { return hash; } DataTypes FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) @@ -57,12 +59,13 @@ SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) } -FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) +FutureSetFromTuple::FutureSetFromTuple( + Hash hash_, Block block, + bool transform_null_in, SizeLimits size_limits) + : hash(hash_) { - auto size_limits = getSizeLimitsForSet(settings); - set = std::make_shared(size_limits, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); + set = std::make_shared(size_limits, 0, transform_null_in); set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - Columns columns; columns.reserve(block.columns()); for (const auto & column : block) @@ -72,9 +75,27 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set->insertFromColumns(columns, set_key_columns); set->finishInsert(); + + for (const auto & type : set->getElementsTypes()) + { + auto name = type->getName(); + hash = CityHash_v1_0_2::CityHash128WithSeed(name.data(), name.size(), hash); + } } DataTypes FutureSetFromTuple::getTypes() const { return set->getElementsTypes(); } +FutureSet::Hash FutureSetFromTuple::getHash() const { return hash; } + +Columns FutureSetFromTuple::getKeyColumns() +{ + if (!set->hasExplicitSetElements()) + { + set->fillSetElements(); + set->appendSetElements(set_key_columns); + } + + return set->getSetElements(); +} SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { @@ -95,32 +116,33 @@ SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) FutureSetFromSubquery::FutureSetFromSubquery( - String key, + Hash hash_, std::unique_ptr source_, StoragePtr external_table_, std::shared_ptr external_table_set_, - const Settings & settings) - : external_table(std::move(external_table_)), external_table_set(std::move(external_table_set_)), source(std::move(source_)) + bool transform_null_in, + SizeLimits size_limits, + size_t max_size_for_index) + : hash(hash_), external_table(std::move(external_table_)), external_table_set(std::move(external_table_set_)), source(std::move(source_)) { set_and_key = std::make_shared(); - set_and_key->key = std::move(key); + set_and_key->key = PreparedSets::toString(hash_, {}); - auto size_limits = getSizeLimitsForSet(settings); - set_and_key->set = std::make_shared(size_limits, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); + set_and_key->set = std::make_shared(size_limits, max_size_for_index, transform_null_in); set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); } FutureSetFromSubquery::FutureSetFromSubquery( - String key, + Hash hash_, QueryTreeNodePtr query_tree_, - const Settings & settings) - : query_tree(std::move(query_tree_)) + bool transform_null_in, + SizeLimits size_limits, + size_t max_size_for_index) + : hash(hash_), query_tree(std::move(query_tree_)) { set_and_key = std::make_shared(); - set_and_key->key = std::move(key); - - auto size_limits = getSizeLimitsForSet(settings); - set_and_key->set = std::make_shared(size_limits, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); + set_and_key->key = PreparedSets::toString(hash_, {}); + set_and_key->set = std::make_shared(size_limits, max_size_for_index, transform_null_in); } FutureSetFromSubquery::~FutureSetFromSubquery() = default; @@ -133,6 +155,8 @@ SetPtr FutureSetFromSubquery::get() const return nullptr; } +SetPtr FutureSetFromSubquery::getNotFilled() const { return set_and_key->set; } + void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source_) { source = std::move(source_); @@ -144,6 +168,8 @@ DataTypes FutureSetFromSubquery::getTypes() const return set_and_key->set->getElementsTypes(); } +FutureSet::Hash FutureSetFromSubquery::getHash() const { return hash; } + std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) { if (set_and_key->set->isCreated()) @@ -252,7 +278,11 @@ String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & FutureSetFromTuplePtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) { - auto from_tuple = std::make_shared(std::move(block), settings); + auto size_limits = getSizeLimitsForSet(settings); + auto from_tuple = std::make_shared( + key, std::move(block), + settings.transform_null_in, size_limits); + const auto & set_types = from_tuple->getTypes(); auto & sets_by_hash = sets_from_tuple[key]; @@ -264,9 +294,9 @@ FutureSetFromTuplePtr PreparedSets::addFromTuple(const Hash & key, Block block, return from_tuple; } -FutureSetFromStoragePtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) +FutureSetFromStoragePtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_, StorageID storage_id) { - auto from_storage = std::make_shared(std::move(set_)); + auto from_storage = std::make_shared(key, std::move(set_), std::move(storage_id)); auto [it, inserted] = sets_from_storage.emplace(key, from_storage); if (!inserted) @@ -282,8 +312,10 @@ FutureSetFromSubqueryPtr PreparedSets::addFromSubquery( FutureSetFromSubqueryPtr external_table_set, const Settings & settings) { + auto size_limits = getSizeLimitsForSet(settings); auto from_subquery = std::make_shared( - toString(key, {}), std::move(source), std::move(external_table), std::move(external_table_set), settings); + key, std::move(source), std::move(external_table), std::move(external_table_set), + settings.transform_null_in, size_limits, settings.use_index_for_in_with_subqueries_max_values); auto [it, inserted] = sets_from_subqueries.emplace(key, from_subquery); @@ -298,10 +330,10 @@ FutureSetFromSubqueryPtr PreparedSets::addFromSubquery( QueryTreeNodePtr query_tree, const Settings & settings) { + auto size_limits = getSizeLimitsForSet(settings); auto from_subquery = std::make_shared( - toString(key, {}), - std::move(query_tree), - settings); + key, std::move(query_tree), + settings.transform_null_in, size_limits, settings.use_index_for_in_with_subqueries_max_values); auto [it, inserted] = sets_from_subqueries.emplace(key, from_subquery); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index a6aee974d0e..750bdfe3189 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -9,6 +9,8 @@ #include #include #include +#include +#include namespace DB { @@ -50,6 +52,9 @@ public: virtual DataTypes getTypes() const = 0; /// If possible, return set with stored elements useful for PK analysis. virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; + + using Hash = CityHash_v1_0_2::uint128; + virtual Hash getHash() const = 0; }; using FutureSetPtr = std::shared_ptr; @@ -59,13 +64,17 @@ using FutureSetPtr = std::shared_ptr; class FutureSetFromStorage final : public FutureSet { public: - explicit FutureSetFromStorage(SetPtr set_); + explicit FutureSetFromStorage(Hash hash_, SetPtr set_, std::optional storage_id); SetPtr get() const override; DataTypes getTypes() const override; SetPtr buildOrderedSetInplace(const ContextPtr &) override; + Hash getHash() const override; + const std::optional & getStorageID() const { return storage_id; } private: + Hash hash; + std::optional storage_id; SetPtr set; }; @@ -76,14 +85,16 @@ using FutureSetFromStoragePtr = std::shared_ptr; class FutureSetFromTuple final : public FutureSet { public: - FutureSetFromTuple(Block block, const Settings & settings); + FutureSetFromTuple(Hash hash_, Block block, bool transform_null_in, SizeLimits size_limits); SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; DataTypes getTypes() const override; - + Hash getHash() const override; + Columns getKeyColumns(); private: + Hash hash; SetPtr set; SetKeyColumns set_key_columns; }; @@ -106,21 +117,27 @@ class FutureSetFromSubquery final : public FutureSet { public: FutureSetFromSubquery( - String key, + Hash hash_, std::unique_ptr source_, StoragePtr external_table_, std::shared_ptr external_table_set_, - const Settings & settings); + bool transform_null_in, + SizeLimits size_limits, + size_t max_size_for_index); FutureSetFromSubquery( - String key, + Hash hash_, QueryTreeNodePtr query_tree_, - const Settings & settings); + bool transform_null_in, + SizeLimits size_limits, + size_t max_size_for_index); ~FutureSetFromSubquery() override; SetPtr get() const override; + SetPtr getNotFilled() const; DataTypes getTypes() const override; + Hash getHash() const override; SetPtr buildOrderedSetInplace(const ContextPtr & context) override; std::unique_ptr build(const ContextPtr & context); @@ -129,7 +146,11 @@ public: QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); } void setQueryPlan(std::unique_ptr source_); + const QueryPlan * getQueryPlan() const { return source.get(); } + QueryPlan * getQueryPlan() { return source.get(); } + private: + Hash hash; SetAndKeyPtr set_and_key; StoragePtr external_table; std::shared_ptr external_table_set; @@ -155,7 +176,7 @@ public: using SetsFromStorage = std::unordered_map; using SetsFromSubqueries = std::unordered_map; - FutureSetFromStoragePtr addFromStorage(const Hash & key, SetPtr set_); + FutureSetFromStoragePtr addFromStorage(const Hash & key, SetPtr set_, StorageID storage_id); FutureSetFromTuplePtr addFromTuple(const Hash & key, Block block, const Settings & settings); FutureSetFromSubqueryPtr addFromSubquery( @@ -184,6 +205,8 @@ public: static String toString(const Hash & key, const DataTypes & types); + static SizeLimits getSizeLimitsForSet(const Settings & settings); + private: SetsFromTuple sets_from_tuple; SetsFromStorage sets_from_storage; diff --git a/src/Interpreters/SelectQueryOptions.h b/src/Interpreters/SelectQueryOptions.h index efe94f96772..e48e9f59c4b 100644 --- a/src/Interpreters/SelectQueryOptions.h +++ b/src/Interpreters/SelectQueryOptions.h @@ -56,6 +56,9 @@ struct SelectQueryOptions std::optional shard_num; std::optional shard_count; + bool build_logical_plan = false; + bool ignore_rename_columns = false; + /** During read from MergeTree parts will be removed from snapshot after they are not needed. * This optimization will break subsequent execution of the same query tree, because table node * will no more have valid snapshot. diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 8b44f36b278..440cb975bbf 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -33,9 +33,8 @@ public: /// store all set elements in explicit form. /// This is needed for subsequent use for index. Set(const SizeLimits & limits_, size_t max_elements_to_fill_, bool transform_null_in_) - : log(getLogger("Set")), - limits(limits_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_), - cast_cache(std::make_unique()) + : limits(limits_), transform_null_in(transform_null_in_), max_elements_to_fill(max_elements_to_fill_) + , log(getLogger("Set")), cast_cache(std::make_unique()) {} /** Set can be created either from AST or from a stream of data (subquery result). @@ -86,6 +85,14 @@ public: static DataTypes getElementTypes(DataTypes types, bool transform_null_in); + /// Limitations on the maximum size of the set + const SizeLimits limits; + + /// If true, insert NULL values to set. + const bool transform_null_in; + + const size_t max_elements_to_fill; + private: size_t keys_size = 0; Sizes key_sizes; @@ -117,15 +124,8 @@ private: LoggerPtr log; - /// Limitations on the maximum size of the set - SizeLimits limits; - /// Do we need to additionally store all elements of the set in explicit form for subsequent use for index. bool fill_set_elements = false; - size_t max_elements_to_fill; - - /// If true, insert NULL values to set. - bool transform_null_in; /// Check if set contains all the data. std::atomic is_created = false; diff --git a/src/Interpreters/SetSerialization.h b/src/Interpreters/SetSerialization.h new file mode 100644 index 00000000000..1413aa556da --- /dev/null +++ b/src/Interpreters/SetSerialization.h @@ -0,0 +1,32 @@ +#pragma once +#include + +namespace DB +{ + +class FutureSet; +using FutureSetPtr = std::shared_ptr; + +struct SerializedSetsRegistry +{ + struct Hashing + { + UInt64 operator()(const FutureSet::Hash & key) const { return key.low64 ^ key.high64; } + }; + + std::unordered_map sets; +}; + +class ColumnSet; + +struct DeserializedSetsRegistry +{ + struct Hashing + { + UInt64 operator()(const FutureSet::Hash & key) const { return key.low64 ^ key.high64; } + }; + + std::unordered_map, Hashing> sets; +}; + +} diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3260ea890c6..b8c72550bf2 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -74,6 +74,9 @@ #include #include #include +#include +#include +#include #include #include @@ -109,6 +112,7 @@ namespace ErrorCodes extern const int SYNTAX_ERROR; extern const int SUPPORT_IS_DISABLED; extern const int INCORRECT_QUERY; + extern const int INCORRECT_DATA; } namespace FailPoints @@ -1408,8 +1412,320 @@ static std::tuple executeQueryImpl( } +static std::pair executeQueryImpl( + ContextMutablePtr context, + QueryFlags flags, + const String & query, + const std::shared_ptr & query_plan) +{ + /// Used for logging query start time in system.query_log + auto query_start_time = std::chrono::system_clock::now(); + + /// Used for: + /// * Setting the watch in QueryStatus (controls timeouts and progress) and the output formats + /// * Logging query duration (system.query_log) + Stopwatch start_watch{CLOCK_MONOTONIC}; + + const Settings & settings = context->getSettingsRef(); + + String query_for_logging; + ASTPtr ast; + const bool internal = flags.internal; + size_t log_queries_cut_to_length = context->getSettingsRef().log_queries_cut_to_length; + + /// query_span is a special span, when this function exits, it's lifetime is not ended, but ends when the query finishes. + /// Some internal queries might call this function recursively by setting 'internal' parameter to 'true', + /// to make sure SpanHolders in current stack ends in correct order, we disable this span for these internal queries + /// + /// This does not have impact on the final span logs, because these internal queries are issued by external queries, + /// we still have enough span logs for the execution of external queries. + std::shared_ptr query_span = internal ? nullptr : std::make_shared("query"); + if (query_span && query_span->trace_id != UUID{}) + LOG_TRACE(getLogger("executeQuery"), "Query span trace_id for opentelemetry log: {}", query_span->trace_id); + + try + { + ParserQuery parser(query.data() + query.size(), settings.allow_settings_after_format_in_insert); + /// TODO: parser should fail early when max_query_size limit is reached. + ast = parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth, settings.max_parser_backtracks); + + /// Wipe any sensitive information (e.g. passwords) from the query. + /// MUST go before any modification (except for prepared statements, + /// since it substitute parameters and without them query does not contain + /// parameters), to keep query as-is in query_log and server log. + if (ast->hasSecretParts()) + { + /// IAST::formatForLogging() wipes secret parts in AST and then calls wipeSensitiveDataAndCutToLength(). + query_for_logging = ast->formatForLogging(log_queries_cut_to_length); + } + else + { + query_for_logging = wipeSensitiveDataAndCutToLength(query, log_queries_cut_to_length); + } + } + catch (...) + { + query_for_logging = wipeSensitiveDataAndCutToLength(query, log_queries_cut_to_length); + logQuery(query_for_logging, context, internal, QueryProcessingStage::QueryPlan); + + if (!internal) + logExceptionBeforeStart(query_for_logging, context, ast, query_span, start_watch.elapsedMilliseconds()); + throw; + } + + const auto & client_info = context->getClientInfo(); + + if (!internal && client_info.initial_query_start_time == 0) + { + // If it's not an internal query and we don't see an initial_query_start_time yet, initialize it + // to current time. Internal queries are those executed without an independent client context, + // thus should not set initial_query_start_time, because it might introduce data race. It's also + // possible to have unset initial_query_start_time for non-internal and non-initial queries. For + // example, the query is from an initiator that is running an old version of clickhouse. + // On the other hand, if it's initialized then take it as the start of the query + context->setInitialQueryStartTime(query_start_time); + } + + assert(internal || CurrentThread::get().getQueryContext()); + assert(internal || CurrentThread::get().getQueryContext()->getCurrentQueryId() == CurrentThread::getQueryId()); + + /// Avoid early destruction of process_list_entry if it was not saved to `res` yet (in case of exception) + ProcessList::EntryPtr process_list_entry; + BlockIO res; + auto implicit_txn_control = std::make_shared(false); + String query_database; + String query_table; + + auto execute_implicit_tcl_query = [implicit_txn_control](const ContextMutablePtr & query_context, ASTTransactionControl::QueryType tcl_type) + { + /// Unset the flag on COMMIT and ROLLBACK + SCOPE_EXIT({ if (tcl_type != ASTTransactionControl::BEGIN) *implicit_txn_control = false; }); + + ASTPtr tcl_ast = std::make_shared(tcl_type); + InterpreterTransactionControlQuery tc(tcl_ast, query_context); + tc.execute(); + + /// Set the flag after successful BIGIN + if (tcl_type == ASTTransactionControl::BEGIN) + *implicit_txn_control = true; + }; + + try + { + if (auto txn = context->getCurrentTransaction()) + { + chassert(txn->getState() != MergeTreeTransaction::COMMITTING); + chassert(txn->getState() != MergeTreeTransaction::COMMITTED); + if (txn->getState() == MergeTreeTransaction::ROLLED_BACK) + throw Exception( + ErrorCodes::INVALID_TRANSACTION, + "Cannot execute query because current transaction failed. Expecting ROLLBACK statement"); + } + + logQuery({}, context, internal, QueryProcessingStage::QueryPlan); + + /// Put query to process list. But don't put SHOW PROCESSLIST query itself. + if (!internal) + { + /// processlist also has query masked now, to avoid secrets leaks though SHOW PROCESSLIST by other users. + process_list_entry = context->getProcessList().insert(query_for_logging, ast.get(), context, start_watch.getStart()); + context->setProcessListElement(process_list_entry->getQueryStatus()); + } + + /// Load external tables if they were provided + context->initializeExternalTablesIfSet(); + if (!query_plan) + throw Exception(ErrorCodes::INCORRECT_DATA, "Expected query plan packet for QueryPlan stage"); + + /// reset Input callbacks if query is not INSERT SELECT + context->resetInputCallbacks(); + + StreamLocalLimits limits; + std::shared_ptr quota; + + auto logger = getLogger("executeQuery"); + + bool quota_checked = false; + std::unique_ptr insert_data_buffer_holder; + + if (true) + { + if (true) + { + /// We need to start the (implicit) transaction before getting the interpreter as this will get links to the latest snapshots + if (!context->getCurrentTransaction() && settings.implicit_transaction) + { + try + { + if (context->isGlobalContext()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot create transactions"); + + execute_implicit_tcl_query(context, ASTTransactionControl::BEGIN); + } + catch (Exception & e) + { + e.addMessage("while starting a transaction with 'implicit_transaction'"); + throw; + } + } + + const auto & query_settings = context->getSettingsRef(); + if (context->getCurrentTransaction() && query_settings.throw_on_unsupported_query_inside_transaction) + { + //if (!interpreter->supportsTransactions()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query (QueryPlan)"); + + } + + if (!quota_checked) + { + quota = context->getQuota(); + if (quota) + { + // if (ast->as() || ast->as()) + { + quota->used(QuotaType::QUERY_SELECTS, 1); + } + // else if (ast->as()) + // { + // quota->used(QuotaType::QUERY_INSERTS, 1); + // } + quota->used(QuotaType::QUERIES, 1); + quota->checkExceeded(QuotaType::ERRORS); + } + } + + if (true) + { + limits.mode = LimitsMode::LIMITS_CURRENT; + limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); + } + + { + std::unique_ptr span; + if (OpenTelemetry::CurrentContext().isTraceEnabled()) + { + // auto * raw_interpreter_ptr = interpreter.get(); + // String class_name(demangle(typeid(*raw_interpreter_ptr).name())); + span = std::make_unique("QueryPlan::execute()"); + } + + auto plan = QueryPlan::resolveStorages(std::move(*query_plan), context); + + WriteBufferFromOwnString buf; + plan.explainPlan(buf, {.header=true, .actions=true}); + LOG_TRACE(getLogger("executeQuery"), "Query Plan:\n{}", buf.str()); + + auto pipeline = plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(context), + BuildQueryPipelineSettings::fromContext(context)); + + res.pipeline = QueryPipelineBuilder::getPipeline(std::move(*pipeline)); + + } + } + } + + if (process_list_entry) + { + /// Query was killed before execution + if (process_list_entry->getQueryStatus()->isKilled()) + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, + "Query '{}' is killed in pending state", process_list_entry->getQueryStatus()->getInfo().client_info.current_query_id); + } + + /// Hold element of process list till end of query execution. + res.process_list_entry = process_list_entry; + + auto & pipeline = res.pipeline; + + if (pipeline.pulling() || pipeline.completed()) + { + /// Limits on the result, the quota on the result, and also callback for progress. + /// Limits apply only to the final result. + pipeline.setProgressCallback(context->getProgressCallback()); + pipeline.setProcessListElement(context->getProcessListElement()); + // if (stage == QueryProcessingStage::Complete && pipeline.pulling()) + // pipeline.setLimitsAndQuota(limits, quota); + } + else if (pipeline.pushing()) + { + pipeline.setProcessListElement(context->getProcessListElement()); + } + + /// Everything related to query log. + { + QueryLogElement elem = logQueryStart( + query_start_time, + context, + query_for_logging, + ast, + pipeline, + nullptr, + internal, + query_database, + query_table, + false); + /// Also make possible for caller to log successful query finish and exception during execution. + auto finish_callback = [elem, + context, + ast, + //query_cache_usage, + internal, + implicit_txn_control, + execute_implicit_tcl_query, + pulling_pipeline = pipeline.pulling(), + query_span](QueryPipeline & query_pipeline) mutable + { + // if (query_cache_usage == QueryCache::Usage::Write) + // /// Trigger the actual write of the buffered query result into the query cache. This is done explicitly to prevent + // /// partial/garbage results in case of exceptions during query execution. + // query_pipeline.finalizeWriteInQueryCache(); + + logQueryFinish(elem, context, ast, query_pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); + + if (*implicit_txn_control) + execute_implicit_tcl_query(context, ASTTransactionControl::COMMIT); + }; + + auto exception_callback = + [start_watch, elem, context, ast, internal, my_quota(quota), implicit_txn_control, execute_implicit_tcl_query, query_span]( + bool log_error) mutable + { + if (*implicit_txn_control) + execute_implicit_tcl_query(context, ASTTransactionControl::ROLLBACK); + else if (auto txn = context->getCurrentTransaction()) + txn->onException(); + + if (my_quota) + my_quota->used(QuotaType::ERRORS, 1, /* check_exceeded = */ false); + + logQueryException(elem, context, start_watch, ast, query_span, internal, log_error); + }; + + res.finish_callback = std::move(finish_callback); + res.exception_callback = std::move(exception_callback); + } + } + catch (...) + { + if (*implicit_txn_control) + execute_implicit_tcl_query(context, ASTTransactionControl::ROLLBACK); + else if (auto txn = context->getCurrentTransaction()) + txn->onException(); + + if (!internal) + logExceptionBeforeStart(query_for_logging, context, ast, query_span, start_watch.elapsedMilliseconds()); + + throw; + } + + return std::make_tuple(std::move(ast), std::move(res)); +} + std::pair executeQuery( const String & query, + const std::shared_ptr & query_plan, ContextMutablePtr context, QueryFlags flags, QueryProcessingStage::Enum stage) @@ -1417,7 +1733,10 @@ std::pair executeQuery( ASTPtr ast; BlockIO res; - std::tie(ast, res) = executeQueryImpl(query.data(), query.data() + query.size(), context, flags, stage, nullptr); + if (stage == QueryProcessingStage::QueryPlan) + std::tie(ast, res) = executeQueryImpl(context, flags, query, query_plan); + else + std::tie(ast, res) = executeQueryImpl(query.data(), query.data() + query.size(), context, flags, stage, nullptr); if (const auto * ast_query_with_output = dynamic_cast(ast.get())) { diff --git a/src/Interpreters/executeQuery.h b/src/Interpreters/executeQuery.h index c6b3e1fc34e..1c2bf290372 100644 --- a/src/Interpreters/executeQuery.h +++ b/src/Interpreters/executeQuery.h @@ -17,6 +17,7 @@ class ReadBuffer; class WriteBuffer; class IOutputFormat; struct QueryStatusInfo; +struct QueryPlanAndSets; struct QueryResultDetails { @@ -64,7 +65,8 @@ void executeQuery( /// Correctly formatting the results (according to INTO OUTFILE and FORMAT sections) /// must be done separately. std::pair executeQuery( - const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out. + const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out. + const std::shared_ptr & query_plan, ContextMutablePtr context, /// DB, tables, data types, storage engines, functions, aggregate functions... QueryFlags flags = {}, QueryProcessingStage::Enum stage = QueryProcessingStage::Complete /// To which stage the query must be executed. diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index cf18cbbb54a..a4c227fb25a 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -158,6 +158,8 @@ Block getHeaderForProcessingStage( return result; } + case QueryProcessingStage::QueryPlan: + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get header for QueryPlan stage."); } throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown processed stage."); } diff --git a/src/Interpreters/loadMetadata.cpp b/src/Interpreters/loadMetadata.cpp index 4d17891f9f8..3a2c0bd5d25 100644 --- a/src/Interpreters/loadMetadata.cpp +++ b/src/Interpreters/loadMetadata.cpp @@ -306,7 +306,7 @@ static void convertOrdinaryDatabaseToAtomic(LoggerPtr log, ContextMutablePtr con LOG_INFO(log, "Will convert database {} from Ordinary to Atomic", name_quoted); String create_database_query = fmt::format("CREATE DATABASE IF NOT EXISTS {}", tmp_name_quoted); - auto res = executeQuery(create_database_query, context, QueryFlags{ .internal = true }).second; + auto res = executeQuery(create_database_query, nullptr, context, QueryFlags{ .internal = true }).second; executeTrivialBlockIO(res, context); res = {}; auto tmp_database = DatabaseCatalog::instance().getDatabase(tmp_name); @@ -346,7 +346,7 @@ static void convertOrdinaryDatabaseToAtomic(LoggerPtr log, ContextMutablePtr con String tmp_qualified_quoted_name = id.getFullTableName(); String move_table_query = fmt::format("RENAME TABLE {} TO {}", qualified_quoted_name, tmp_qualified_quoted_name); - res = executeQuery(move_table_query, context, QueryFlags{ .internal = true }).second; + res = executeQuery(move_table_query, nullptr, context, QueryFlags{ .internal = true }).second; executeTrivialBlockIO(res, context); res = {}; } @@ -358,12 +358,12 @@ static void convertOrdinaryDatabaseToAtomic(LoggerPtr log, ContextMutablePtr con String drop_query = fmt::format("DROP DATABASE {}", name_quoted); context->setSetting("force_remove_data_recursively_on_drop", false); - res = executeQuery(drop_query, context, QueryFlags{ .internal = true }).second; + res = executeQuery(drop_query, nullptr, context, QueryFlags{ .internal = true }).second; executeTrivialBlockIO(res, context); res = {}; String rename_query = fmt::format("RENAME DATABASE {} TO {}", tmp_name_quoted, name_quoted); - res = executeQuery(rename_query, context, QueryFlags{ .internal = true }).second; + res = executeQuery(rename_query, nullptr, context, QueryFlags{ .internal = true }).second; executeTrivialBlockIO(res, context); LOG_INFO(log, "Finished database engine conversion of {}", name_quoted); @@ -433,7 +433,7 @@ static void maybeConvertOrdinaryDatabaseToAtomic(ContextMutablePtr context, cons /// Reload database just in case (and update logger name) String detach_query = fmt::format("DETACH DATABASE {}", backQuoteIfNeed(database_name)); - auto res = executeQuery(detach_query, context, QueryFlags{ .internal = true }).second; + auto res = executeQuery(detach_query, nullptr, context, QueryFlags{ .internal = true }).second; executeTrivialBlockIO(res, context); res = {}; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 9cac4e61380..65b2ec323ac 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -65,7 +65,7 @@ public: if (sets.findStorage(set_key)) return; - sets.addFromStorage(set_key, storage_set->getSet()); + sets.addFromStorage(set_key, storage_set->getSet(), second_argument_table->getStorageID()); } else if (const auto * constant_node = in_second_argument->as()) { diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 43177fc73c0..4f56c545977 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -747,12 +747,9 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi lambda_actions_dag.getOutputs().push_back(actions_stack.back().getNodeOrThrow(lambda_expression_node_name)); lambda_actions_dag.removeUnusedActions(Names(1, lambda_expression_node_name)); - auto expression_actions_settings = ExpressionActionsSettings::fromContext(planner_context->getQueryContext(), CompileExpressions::yes); - auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); - Names captured_column_names; ActionsDAG::NodeRawConstPtrs lambda_children; - Names required_column_names = lambda_actions->getRequiredColumns(); + Names required_column_names = lambda_actions_dag.getRequiredColumnsNames(); actions_stack.pop_back(); levels.reset(actions_stack.size()); @@ -773,7 +770,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi auto lambda_node_name = calculateActionNodeName(node, *planner_context); auto function_capture = std::make_shared( - lambda_actions, captured_column_names, lambda_arguments_names_and_types, lambda_node.getExpression()->getResultType(), lambda_expression_node_name); + std::make_shared(std::move(lambda_actions_dag)), captured_column_names, lambda_arguments_names_and_types, lambda_node.getExpression()->getResultType(), lambda_expression_node_name); // TODO: Pass IFunctionBase here not FunctionCaptureOverloadResolver. const auto * actions_node = actions_stack[level].addFunctionIfNecessary(lambda_node_name, std::move(lambda_children), function_capture); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index e0668a0eb8e..1e042765c3f 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -48,6 +48,8 @@ #include #include #include +#include +#include #include #include @@ -754,7 +756,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } /// Apply trivial_count optimization if possible - bool is_trivial_count_applied = !select_query_options.only_analyze && is_single_table_expression + bool is_trivial_count_applied = !select_query_options.only_analyze && !select_query_options.build_logical_plan && is_single_table_expression && (table_node || table_function_node) && select_query_info.has_aggregates && settings.additional_table_filters.value.empty() && applyTrivialCountIfPossible( query_plan, @@ -776,7 +778,18 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres auto & prewhere_info = table_expression_query_info.prewhere_info; const auto & prewhere_actions = table_expression_data.getPrewhereFilterActions(); - if (prewhere_actions) + std::vector> where_filters; + + if (prewhere_actions && select_query_options.build_logical_plan) + { + where_filters.emplace_back( + FilterDAGInfo{ + prewhere_actions->clone(), + prewhere_actions->getOutputs().at(0)->result_name, + true}, + "Prewhere"); + } + else if (prewhere_actions) { prewhere_info = std::make_shared(); prewhere_info->prewhere_actions = prewhere_actions->clone(); @@ -789,7 +802,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres const auto & columns_names = table_expression_data.getColumnNames(); - std::vector> where_filters; const auto add_filter = [&](FilterDAGInfo & filter_info, std::string description) { bool is_final = table_expression_query_info.table_expression_modifiers @@ -798,7 +810,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres = settings.optimize_move_to_prewhere && (!is_final || settings.optimize_move_to_prewhere_if_final); auto supported_prewhere_columns = storage->supportedPrewhereColumns(); - if (storage->canMoveConditionsToPrewhere() && optimize_move_to_prewhere && (!supported_prewhere_columns || supported_prewhere_columns->contains(filter_info.column_name))) + if (!select_query_options.build_logical_plan && storage->canMoveConditionsToPrewhere() && optimize_move_to_prewhere && (!supported_prewhere_columns || supported_prewhere_columns->contains(filter_info.column_name))) { if (!prewhere_info) { @@ -857,8 +869,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context)) add_filter(*additional_filters_info, "additional filter"); - from_stage = storage->getQueryProcessingStage( - query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); + if (!select_query_options.build_logical_plan) + from_stage = storage->getQueryProcessingStage( + query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); /// It is just a safety check needed until we have a proper sending plan to replicas. /// If we have a non-trivial storage like View it might create its own Planner inside read(), run findTableForParallelReplicas() @@ -869,15 +882,49 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (other_table_already_chosen_for_reading_with_parallel_replicas) planner_context->getMutableQueryContext()->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); - storage->read( - query_plan, - columns_names, - storage_snapshot, - table_expression_query_info, - query_context, - from_stage, - max_block_size, - max_streams); + if (select_query_options.build_logical_plan) + { + auto sample_block = storage_snapshot->getSampleBlockForColumns(columns_names); + + if (table_node) + { + String table_name; + if (!table_node->getTemporaryTableName().empty()) + table_name = table_node->getTemporaryTableName(); + else + table_name = table_node->getStorageID().getFullTableName(); + + auto reading_from_table = std::make_unique( + sample_block, + table_name, + table_expression_query_info.table_expression_modifiers.value_or(TableExpressionModifiers{})); + + query_plan.addStep(std::move(reading_from_table)); + } + else if (table_function_node) + { + auto table_function_ast = table_function_node->toAST(); + table_function_ast->setAlias({}); + auto table_function_serialized_ast = queryToString(table_function_ast); + + auto reading_from_table_function = std::make_unique( + sample_block, + table_function_serialized_ast, + table_expression_query_info.table_expression_modifiers.value_or(TableExpressionModifiers{})); + + query_plan.addStep(std::move(reading_from_table_function)); + } + } + else + storage->read( + query_plan, + columns_names, + storage_snapshot, + table_expression_query_info, + query_context, + from_stage, + max_block_size, + max_streams); auto parallel_replicas_enabled_for_storage = [](const StoragePtr & table, const Settings & query_settings) { @@ -891,7 +938,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres }; /// query_plan can be empty if there is nothing to read - if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings)) + if (query_plan.isInitialized() && !select_query_options.build_logical_plan && parallel_replicas_enabled_for_storage(storage, settings)) { if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0) { @@ -1115,7 +1162,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres table_expression->formatASTForErrorMessage()); } - if (from_stage == QueryProcessingStage::FetchColumns) + if (from_stage == QueryProcessingStage::FetchColumns && !select_query_options.ignore_rename_columns) { ActionsDAG rename_actions_dag(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); ActionsDAG::NodeRawConstPtrs updated_actions_dag_outputs; @@ -1135,6 +1182,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres rename_step->setStepDescription("Change column names to column identifiers"); query_plan.addStep(std::move(rename_step)); } + else if (from_stage == QueryProcessingStage::FetchColumns) + { + } else { SelectQueryOptions analyze_query_options = SelectQueryOptions(from_stage).analyze(); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index a4d707704b1..3c1bb5ab1de 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -13,7 +13,9 @@ #include #include #include -#include +#include +#include +#include #include #include #include @@ -30,6 +32,8 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; + extern const int INCORRECT_DATA; } static bool memoryBoundMergingWillBeUsed( @@ -151,6 +155,25 @@ void AggregatingStep::applyOrder(SortDescription sort_description_for_merging_, explicit_sorting_required_for_aggregation_in_order = false; } +static void updateThreadsValues( + size_t & new_merge_threads, + size_t & new_temporary_data_merge_threads, + Aggregator::Params & params, + const BuildQueryPipelineSettings & settings) +{ + /// Update values from settings if plan was deserialized. + if (new_merge_threads == 0) + new_merge_threads = settings.max_threads; + + if (new_temporary_data_merge_threads == 0) + new_temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads; + if (new_temporary_data_merge_threads == 0) + new_temporary_data_merge_threads = new_merge_threads; + + if (params.max_threads == 0) + params.max_threads = settings.max_threads; +} + ActionsDAG AggregatingStep::makeCreatingMissingKeysForGroupingSetDAG( const Block & in_header, const Block & out_header, @@ -208,6 +231,10 @@ ActionsDAG AggregatingStep::makeCreatingMissingKeysForGroupingSetDAG( void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) { + size_t new_merge_threads = merge_threads; + size_t new_temporary_data_merge_threads = temporary_data_merge_threads; + updateThreadsValues(new_merge_threads, new_temporary_data_merge_threads, params, settings); + QueryPipelineProcessorsCollector collector(pipeline, this); /// Forget about current totals and extremes. They will be calculated again after aggregation if needed. @@ -306,8 +333,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B transform_params_for_set, many_data, j, - merge_threads, - temporary_data_merge_threads, + new_merge_threads, + new_temporary_data_merge_threads, should_produce_results_in_order_of_bucket_number, skip_merging); // For each input stream we have `grouping_sets_size` copies, so port index @@ -400,7 +427,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B return std::make_shared( header, transform_params, sort_description_for_merging, group_by_sort_description, - max_block_size, aggregation_in_order_max_block_bytes / merge_threads, + max_block_size, aggregation_in_order_max_block_bytes / new_merge_threads, many_data, counter++); }); @@ -426,7 +453,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B pipeline.addTransform(std::move(transform)); /// Do merge of aggregated data in parallel. - pipeline.resize(merge_threads); + pipeline.resize(new_merge_threads); const auto & required_sort_description = memoryBoundMergingWillBeUsed() ? group_by_sort_description : SortDescription{}; pipeline.addSimpleTransform( @@ -482,8 +509,8 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B transform_params, many_data, counter++, - merge_threads, - temporary_data_merge_threads, + new_merge_threads, + new_temporary_data_merge_threads, should_produce_results_in_order_of_bucket_number, skip_merging); }); @@ -616,8 +643,12 @@ AggregatingProjectionStep::AggregatingProjectionStep( QueryPipelineBuilderPtr AggregatingProjectionStep::updatePipeline( QueryPipelineBuilders pipelines, - const BuildQueryPipelineSettings &) + const BuildQueryPipelineSettings & settings) { + size_t new_merge_threads = merge_threads; + size_t new_temporary_data_merge_threads = temporary_data_merge_threads; + updateThreadsValues(new_merge_threads, new_temporary_data_merge_threads, params, settings); + auto & normal_parts_pipeline = pipelines.front(); auto & projection_parts_pipeline = pipelines.back(); @@ -652,7 +683,7 @@ QueryPipelineBuilderPtr AggregatingProjectionStep::updatePipeline( pipeline.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, transform_params, many_data, counter++, merge_threads, temporary_data_merge_threads); + header, transform_params, many_data, counter++, new_merge_threads, new_temporary_data_merge_threads); }); }; @@ -669,4 +700,177 @@ QueryPipelineBuilderPtr AggregatingProjectionStep::updatePipeline( return pipeline; } + +void AggregatingStep::serializeSettings(QueryPlanSerializationSettings & settings) const +{ + settings.max_block_size = max_block_size; + settings.aggregation_in_order_max_block_bytes = aggregation_in_order_max_block_bytes; + + settings.aggregation_in_order_memory_bound_merging = should_produce_results_in_order_of_bucket_number; + settings.aggregation_sort_result_by_bucket_number = memory_bound_merging_of_aggregation_results_enabled; + + settings.max_rows_to_group_by = params.max_rows_to_group_by; + settings.group_by_overflow_mode = params.group_by_overflow_mode; + + settings.group_by_two_level_threshold = params.group_by_two_level_threshold; + settings.group_by_two_level_threshold_bytes = params.group_by_two_level_threshold_bytes; + + settings.max_bytes_before_external_group_by = params.max_bytes_before_external_group_by; + settings.empty_result_for_aggregation_by_empty_set = params.empty_result_for_aggregation_by_empty_set; + + settings.min_free_disk_space_for_temporary_data = params.min_free_disk_space; + + settings.compile_aggregate_expressions = params.compile_aggregate_expressions; + settings.min_count_to_compile_aggregate_expression = params.min_count_to_compile_aggregate_expression; + + settings.enable_software_prefetch_in_aggregation = params.enable_prefetch; + settings.optimize_group_by_constant_keys = params.optimize_group_by_constant_keys; + settings.min_hit_rate_to_use_consecutive_keys_optimization = params.min_hit_rate_to_use_consecutive_keys_optimization; + + settings.collect_hash_table_stats_during_aggregation = params.stats_collecting_params.isCollectionAndUseEnabled(); + settings.max_entries_for_hash_table_stats = params.stats_collecting_params.max_entries_for_hash_table_stats; + settings.max_size_to_preallocate_for_aggregation = params.stats_collecting_params.max_size_to_preallocate; +} + +void AggregatingStep::serialize(Serialization & ctx) const +{ + if (!sort_description_for_merging.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Serialization of AggregatingStep optimized for in-order is not supported."); + + if (!grouping_sets_params.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Serialization of AggregatingStep with grouping sets is not supported."); + + /// If you wonder why something is serialized using settings, and other is serialized using flags, considerations are following: + /// * flags are something that may change data format returning from the step + /// * settings are something which already was in Settings.h and, usually, is passed to Aggregator unchanged + /// Flags `final` and `group_by_use_nulls` change types, and `overflow_row` appends additional block to results. + /// Settings like `max_rows_to_group_by` or `empty_result_for_aggregation_by_empty_set` affect the result, + /// but does not change data format. + /// Overall, the rule is not strict. + + UInt8 flags = 0; + if (final) + flags |= 1; + if (params.overflow_row) + flags |= 2; + if (group_by_use_nulls) + flags |= 4; + if (!grouping_sets_params.empty()) + flags |= 8; + if (explicit_sorting_required_for_aggregation_in_order) + flags |= 16; + /// Ideally, key should be calculated from QueryPlan on the follower. + /// So, let's have a flag to disable sending/reading pre-calculated value. + if (params.stats_collecting_params.isCollectionAndUseEnabled()) + flags |= 32; + + writeIntBinary(flags, ctx.out); + + if (explicit_sorting_required_for_aggregation_in_order) + serializeSortDescription(group_by_sort_description, ctx.out); + + writeVarUInt(params.keys.size(), ctx.out); + for (const auto & key : params.keys) + writeStringBinary(key, ctx.out); + + serializeAggregateDescriptions(params.aggregates, ctx.out); + + if (params.stats_collecting_params.isCollectionAndUseEnabled()) + writeIntBinary(params.stats_collecting_params.key, ctx.out); +} + +std::unique_ptr AggregatingStep::deserialize(Deserialization & ctx) +{ + if (ctx.input_streams.size() != 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "AggregatingStep must have one input stream"); + + UInt8 flags; + readIntBinary(flags, ctx.in); + + bool final = bool(flags & 1); + bool overflow_row = bool(flags & 2); + bool group_by_use_nulls = bool(flags & 4); + bool has_grouping_sets = bool(flags & 8); + bool explicit_sorting_required_for_aggregation_in_order = bool(flags & 16); + bool has_stats_key = bool(flags & 32); + + if (has_grouping_sets) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Serialization of AggregatingStep with grouping sets is not supported."); + + SortDescription group_by_sort_description; + if (explicit_sorting_required_for_aggregation_in_order) + deserializeSortDescription(group_by_sort_description, ctx.in); + + UInt64 num_keys; + readVarUInt(num_keys, ctx.in); + Names keys(num_keys); + for (auto & key : keys) + readStringBinary(key, ctx.in); + + AggregateDescriptions aggregates; + deserializeAggregateDescriptions(aggregates, ctx.in, ctx.input_streams.front().header); + + UInt64 stats_key = 0; + if (has_stats_key) + readIntBinary(stats_key, ctx.in); + + StatsCollectingParams stats_collecting_params( + stats_key, + ctx.settings.collect_hash_table_stats_during_aggregation, + ctx.settings.max_entries_for_hash_table_stats, + ctx.settings.max_size_to_preallocate_for_aggregation); + + Aggregator::Params params + { + keys, + aggregates, + overflow_row, + ctx.settings.max_rows_to_group_by, + ctx.settings.group_by_overflow_mode, + ctx.settings.group_by_two_level_threshold, + ctx.settings.group_by_two_level_threshold_bytes, + ctx.settings.max_bytes_before_external_group_by, + ctx.settings.empty_result_for_aggregation_by_empty_set, + Context::getGlobalContextInstance()->getTempDataOnDisk(), + 0, //settings.max_threads, + ctx.settings.min_free_disk_space_for_temporary_data, + ctx.settings.compile_aggregate_expressions, + ctx.settings.min_count_to_compile_aggregate_expression, + ctx.settings.max_block_size, + ctx.settings.enable_software_prefetch_in_aggregation, + /* only_merge */ false, + ctx.settings.optimize_group_by_constant_keys, + ctx.settings.min_hit_rate_to_use_consecutive_keys_optimization, + stats_collecting_params + }; + + SortDescription sort_description_for_merging; + GroupingSetsParamsList grouping_sets_params; + + auto aggregating_step = std::make_unique( + ctx.input_streams.front(), + std::move(params), + std::move(grouping_sets_params), + final, + ctx.settings.max_block_size, + ctx.settings.aggregation_in_order_max_block_bytes, + 0, //merge_threads, + 0, //temporary_data_merge_threads, + false, // storage_has_evenly_distributed_read, TODO: later + group_by_use_nulls, + std::move(sort_description_for_merging), + std::move(group_by_sort_description), + ctx.settings.aggregation_in_order_memory_bound_merging, + ctx.settings.aggregation_sort_result_by_bucket_number, + explicit_sorting_required_for_aggregation_in_order); + + return aggregating_step; +} + +void registerAggregatingStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("Aggregating", AggregatingStep::deserialize); +} + + } diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 4e4078047f1..9a71752fcb1 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -72,6 +72,11 @@ public: UInt64 group, bool group_by_use_nulls); + void serializeSettings(QueryPlanSerializationSettings & settings) const override; + void serialize(Serialization & ctx) const override; + + static std::unique_ptr deserialize(Deserialization & ctx); + private: void updateOutputStream() override; @@ -96,7 +101,7 @@ private: /// These settings are used to determine if we should resize pipeline to 1 at the end. const bool should_produce_results_in_order_of_bucket_number; - bool memory_bound_merging_of_aggregation_results_enabled; + const bool memory_bound_merging_of_aggregation_results_enabled; bool explicit_sorting_required_for_aggregation_in_order; Processors aggregating_in_order; @@ -118,7 +123,7 @@ public: ); String getName() const override { return "AggregatingProjection"; } - QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) override; + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings & settings) override; private: Aggregator::Params params; diff --git a/src/Processors/QueryPlan/ArrayJoinStep.cpp b/src/Processors/QueryPlan/ArrayJoinStep.cpp index 94cb6ae2ee5..6a1ccdf9ea3 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.cpp +++ b/src/Processors/QueryPlan/ArrayJoinStep.cpp @@ -1,4 +1,7 @@ #include +#include +#include +#include #include #include #include @@ -80,4 +83,50 @@ void ArrayJoinStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Columns", std::move(columns_array)); } +void ArrayJoinStep::serializeSettings(QueryPlanSerializationSettings & settings) const +{ + settings.max_block_size = max_block_size; +} + +void ArrayJoinStep::serialize(Serialization & ctx) const +{ + UInt8 flags = 0; + if (array_join.is_left) + flags |= 1; + if (is_unaligned) + flags |= 2; + + writeIntBinary(flags, ctx.out); + + writeVarUInt(array_join.columns.size(), ctx.out); + for (const auto & column : array_join.columns) + writeStringBinary(column, ctx.out); +} + +std::unique_ptr ArrayJoinStep::deserialize(Deserialization & ctx) +{ + UInt8 flags; + readIntBinary(flags, ctx.in); + + bool is_left = bool(flags & 1); + bool is_unaligned = bool(flags & 2); + + UInt64 num_columns; + readVarUInt(num_columns, ctx.in); + + ArrayJoin array_join; + array_join.is_left = is_left; + array_join.columns.resize(num_columns); + + for (auto & column : array_join.columns) + readStringBinary(column, ctx.in); + + return std::make_unique(ctx.input_streams.front(), std::move(array_join), is_unaligned, ctx.settings.max_block_size); +} + +void registerArrayJoinStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("ArrayJoin", ArrayJoinStep::deserialize); +} + } diff --git a/src/Processors/QueryPlan/ArrayJoinStep.h b/src/Processors/QueryPlan/ArrayJoinStep.h index 1a049d5805e..84f37f8997b 100644 --- a/src/Processors/QueryPlan/ArrayJoinStep.h +++ b/src/Processors/QueryPlan/ArrayJoinStep.h @@ -22,6 +22,11 @@ public: const Names & getColumns() const { return array_join.columns; } bool isLeft() const { return array_join.is_left; } + void serializeSettings(QueryPlanSerializationSettings & settings) const override; + void serialize(Serialization & ctx) const override; + + static std::unique_ptr deserialize(Deserialization & ctx); + private: void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp index fb3ed7f80fc..b89dc2912ff 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.cpp @@ -12,6 +12,10 @@ BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr fr settings.actions_settings = ExpressionActionsSettings::fromSettings(from->getSettingsRef(), CompileExpressions::yes); settings.process_list_element = from->getProcessListElement(); settings.progress_callback = from->getProgressCallback(); + + settings.max_threads = from->getSettingsRef().max_threads; + settings.aggregation_memory_efficient_merge_threads = from->getSettingsRef().aggregation_memory_efficient_merge_threads; + return settings; } diff --git a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h index d99f9a7d1f1..cf913fc1685 100644 --- a/src/Processors/QueryPlan/BuildQueryPipelineSettings.h +++ b/src/Processors/QueryPlan/BuildQueryPipelineSettings.h @@ -22,6 +22,9 @@ struct BuildQueryPipelineSettings ProgressCallback progress_callback = nullptr; TemporaryFileLookupPtr temporary_file_lookup; + size_t max_threads; + size_t aggregation_memory_efficient_merge_threads; + const ExpressionActionsSettings & getActionsSettings() const { return actions_settings; } static BuildQueryPipelineSettings fromContext(ContextPtr from); }; diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 292ec19914c..d518fac6d43 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -63,6 +63,8 @@ public: ContextPtr getContext() const { return context; } PreparedSets::Subqueries detachSets() { return std::move(subqueries); } + void serialize(Serialization &) const override {} + private: PreparedSets::Subqueries subqueries; ContextPtr context; diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index b1c24fc01ce..517007f3361 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -1,4 +1,7 @@ #include +#include +#include +#include #include #include #include @@ -13,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; } static ITransformingStep::Traits getTraits(bool pre_distinct) @@ -62,6 +66,16 @@ DistinctStep::DistinctStep( { } +void DistinctStep::updateLimitHint(UInt64 hint) +{ + if (hint && limit_hint) + /// Both limits are set - take the min + limit_hint = std::min(hint, limit_hint); + else + /// Some limit is not set - take the other one + limit_hint = std::max(hint, limit_hint); +} + void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { if (!pre_distinct) @@ -181,4 +195,58 @@ void DistinctStep::updateOutputStream() getTraits(pre_distinct).data_stream_traits); } +void DistinctStep::serializeSettings(QueryPlanSerializationSettings & settings) const +{ + settings.max_rows_in_distinct = set_size_limits.max_rows; + settings.max_bytes_in_distinct = set_size_limits.max_bytes; + settings.distinct_overflow_mode = set_size_limits.overflow_mode; +} + +void DistinctStep::serialize(Serialization & ctx) const +{ + /// Let's not serialize limit_hint. + /// Ideally, we can get if from a query plan optimization on the follower. + + writeVarUInt(columns.size(), ctx.out); + for (const auto & column : columns) + writeStringBinary(column, ctx.out); +} + +std::unique_ptr DistinctStep::deserialize(Deserialization & ctx, bool pre_distinct_) +{ + if (ctx.input_streams.size() != 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "DistinctStep must have one input stream"); + + size_t columns_size; + readVarUInt(columns_size, ctx.in); + Names column_names(columns_size); + for (size_t i = 0; i < columns_size; ++i) + readStringBinary(column_names[i], ctx.in); + + SizeLimits size_limits; + size_limits.max_rows = ctx.settings.max_rows_in_distinct; + size_limits.max_bytes = ctx.settings.max_bytes_in_distinct; + size_limits.overflow_mode = ctx.settings.distinct_overflow_mode; + + return std::make_unique( + ctx.input_streams.front(), size_limits, 0, column_names, pre_distinct_, false); +} + +std::unique_ptr DistinctStep::deserializeNormal(Deserialization & ctx) +{ + return DistinctStep::deserialize(ctx, false); +} +std::unique_ptr DistinctStep::deserializePre(Deserialization & ctx) +{ + return DistinctStep::deserialize(ctx, true); +} + +void registerDistinctStep(QueryPlanStepRegistry & registry) +{ + /// Preliminary distinct probably can be a query plan optimization. + /// It's easier to serialize it using different names, so that pre-distinct can be potentially removed later. + registry.registerStep("Distinct", DistinctStep::deserializeNormal); + registry.registerStep("PreDistinct", DistinctStep::deserializePre); +} + } diff --git a/src/Processors/QueryPlan/DistinctStep.h b/src/Processors/QueryPlan/DistinctStep.h index d59adbbf92d..c935a7a6912 100644 --- a/src/Processors/QueryPlan/DistinctStep.h +++ b/src/Processors/QueryPlan/DistinctStep.h @@ -20,6 +20,8 @@ public: String getName() const override { return "Distinct"; } const Names & getColumnNames() const { return columns; } + String getSerializationName() const override { return pre_distinct ? "PreDistinct" : "Distinct"; } + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; void describeActions(JSONBuilder::JSONMap & map) const override; @@ -28,6 +30,14 @@ public: bool isPreliminary() const { return pre_distinct; } UInt64 getLimitHint() const { return limit_hint; } + void updateLimitHint(UInt64 hint); + + void serializeSettings(QueryPlanSerializationSettings & settings) const override; + void serialize(Serialization & ctx) const override; + + static std::unique_ptr deserialize(Deserialization & ctx, bool pre_distinct_); + static std::unique_ptr deserializeNormal(Deserialization & ctx); + static std::unique_ptr deserializePre(Deserialization & ctx); private: void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index eb699858bdf..3f6eb9327b6 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -16,7 +16,8 @@ std::unique_ptr createLocalPlan( QueryProcessingStage::Enum processed_stage, size_t shard_num, size_t shard_count, - bool has_missing_objects) + bool has_missing_objects, + bool build_logical_plan) { checkStackSize(); @@ -24,7 +25,7 @@ std::unique_ptr createLocalPlan( auto new_context = Context::createCopy(context); /// Do not push down limit to local plan, as it will break `rows_before_limit_at_least` counter. - if (processed_stage == QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit) + if (!build_logical_plan && processed_stage == QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit) processed_stage = QueryProcessingStage::WithMergeableStateAfterAggregation; /// Do not apply AST optimizations, because query @@ -35,6 +36,8 @@ std::unique_ptr createLocalPlan( .setShardInfo(static_cast(shard_num), static_cast(shard_count)) .ignoreASTOptimizations(); + select_query_options.build_logical_plan = build_logical_plan; + if (context->getSettingsRef().allow_experimental_analyzer) { /// For Analyzer, identifier in GROUP BY/ORDER BY/LIMIT BY lists has been resolved to diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h index f59123a7d88..3721fd259b4 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h @@ -14,5 +14,6 @@ std::unique_ptr createLocalPlan( QueryProcessingStage::Enum processed_stage, size_t shard_num, size_t shard_count, - bool has_missing_objects); + bool has_missing_objects, + bool build_logical_plan = false); } diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 6f88c4527a4..20ab53bb5c2 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include #include @@ -10,6 +12,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + static ITransformingStep::Traits getTraits(const ActionsDAG & actions, const Block & header, const SortDescription & sort_description) { return ITransformingStep::Traits @@ -90,4 +97,23 @@ void ExpressionStep::updateOutputStream() } } +void ExpressionStep::serialize(Serialization & ctx) const +{ + actions_dag.serialize(ctx.out, ctx.registry); +} + +std::unique_ptr ExpressionStep::deserialize(Deserialization & ctx) +{ + ActionsDAG actions_dag = ActionsDAG::deserialize(ctx.in, ctx.registry, ctx.context); + if (ctx.input_streams.size() != 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "ExpressionStep must have one input stream"); + + return std::make_unique(ctx.input_streams.front(), std::move(actions_dag)); +} + +void registerExpressionStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("Expression", ExpressionStep::deserialize); +} + } diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index f2926318cbc..a49b2013e2a 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -25,6 +25,9 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; + void serialize(Serialization & ctx) const override; + static std::unique_ptr deserialize(Deserialization & ctx); + private: void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 0c6b71387b7..2a2294c84ae 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include #include @@ -9,6 +11,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + static ITransformingStep::Traits getTraits(const ActionsDAG & expression, const Block & header, const SortDescription & sort_description, bool remove_filter_column, const String & filter_column_name) { bool preserves_sorting = expression.isSortingPreserved(header, sort_description, remove_filter_column ? filter_column_name : ""); @@ -121,4 +129,39 @@ void FilterStep::updateOutputStream() } } +void FilterStep::serialize(Serialization & ctx) const +{ + UInt8 flags = 0; + if (remove_filter_column) + flags |= 1; + writeIntBinary(flags, ctx.out); + + writeStringBinary(filter_column_name, ctx.out); + + actions_dag.serialize(ctx.out, ctx.registry); +} + +std::unique_ptr FilterStep::deserialize(Deserialization & ctx) +{ + if (ctx.input_streams.size() != 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "FilterStep must have one input stream"); + + UInt8 flags; + readIntBinary(flags, ctx.in); + + bool remove_filter_column = bool(flags & 1); + + String filter_column_name; + readStringBinary(filter_column_name, ctx.in); + + ActionsDAG actions_dag = ActionsDAG::deserialize(ctx.in, ctx.registry, ctx.context); + + return std::make_unique(ctx.input_streams.front(), std::move(actions_dag), std::move(filter_column_name), remove_filter_column); +} + +void registerFilterStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("Filter", FilterStep::deserialize); +} + } diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index b5a31bef5fc..1c907cab9d8 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -26,6 +26,10 @@ public: const String & getFilterColumnName() const { return filter_column_name; } bool removesFilterColumn() const { return remove_filter_column; } + void serialize(Serialization & ctx) const override; + + static std::unique_ptr deserialize(Deserialization & ctx); + private: void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/IQueryPlanStep.cpp b/src/Processors/QueryPlan/IQueryPlanStep.cpp index 10352b330af..99af174f5af 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.cpp +++ b/src/Processors/QueryPlan/IQueryPlanStep.cpp @@ -8,6 +8,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; } const DataStream & IQueryPlanStep::getOutputStream() const @@ -118,4 +119,11 @@ void IQueryPlanStep::appendExtraProcessors(const Processors & extra_processors) processors.insert(processors.end(), extra_processors.begin(), extra_processors.end()); } +void IQueryPlanStep::serialize(Serialization & /*ctx*/) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method serialize is not implemented for {}", getName()); +} + +void IQueryPlanStep::updateOutputStream() { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented"); } + } diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index 44eb7ea0c59..58e700f204d 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -16,11 +16,6 @@ using Processors = std::vector; namespace JSONBuilder { class JSONMap; } -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - /// Description of data stream. /// Single logical data stream may relate to many ports of pipeline. class DataStream @@ -63,6 +58,8 @@ using DataStreams = std::vector; class QueryPlan; using QueryPlanRawPtrs = std::list; +struct QueryPlanSerializationSettings; + /// Single step of query plan. class IQueryPlanStep { @@ -70,6 +67,7 @@ public: virtual ~IQueryPlanStep() = default; virtual String getName() const = 0; + virtual String getSerializationName() const { return getName(); } /// Add processors from current step to QueryPipeline. /// Calling this method, we assume and don't check that: @@ -88,6 +86,12 @@ public: const std::string & getStepDescription() const { return step_description; } void setStepDescription(std::string description) { step_description = std::move(description); } + struct Serialization; + struct Deserialization; + + virtual void serializeSettings(QueryPlanSerializationSettings & /*settings*/) const {} + virtual void serialize(Serialization & /*ctx*/) const; + struct FormatSettings { WriteBuffer & out; @@ -136,7 +140,7 @@ public: virtual bool canUpdateInputStream() const { return false; } protected: - virtual void updateOutputStream() { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented"); } + virtual void updateOutputStream(); DataStreams input_streams; std::optional output_stream; diff --git a/src/Processors/QueryPlan/LimitByStep.cpp b/src/Processors/QueryPlan/LimitByStep.cpp index 8b4abecc12c..33e75d1b340 100644 --- a/src/Processors/QueryPlan/LimitByStep.cpp +++ b/src/Processors/QueryPlan/LimitByStep.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include #include @@ -24,11 +26,11 @@ static ITransformingStep::Traits getTraits() LimitByStep::LimitByStep( const DataStream & input_stream_, - size_t group_length_, size_t group_offset_, const Names & columns_) + size_t group_length_, size_t group_offset_, Names columns_) : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , group_length(group_length_) , group_offset(group_offset_) - , columns(columns_) + , columns(std::move(columns_)) { } @@ -83,4 +85,37 @@ void LimitByStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Offset", group_offset); } +void LimitByStep::serialize(Serialization & ctx) const +{ + writeVarUInt(group_length, ctx.out); + writeVarUInt(group_offset, ctx.out); + + + writeVarUInt(columns.size(), ctx.out); + for (const auto & column : columns) + writeStringBinary(column, ctx.out); +} + +std::unique_ptr LimitByStep::deserialize(Deserialization & ctx) +{ + UInt64 group_length; + UInt64 group_offset; + + readVarUInt(group_length, ctx.in); + readVarUInt(group_offset, ctx.in); + + UInt64 num_columns; + readVarUInt(num_columns, ctx.in); + Names columns(num_columns); + for (auto & column : columns) + readStringBinary(column, ctx.in); + + return std::make_unique(ctx.input_streams.front(), group_length, group_offset, std::move(columns)); +} + +void registerLimitByStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("LimitBy", LimitByStep::deserialize); +} + } diff --git a/src/Processors/QueryPlan/LimitByStep.h b/src/Processors/QueryPlan/LimitByStep.h index 0edda3247d6..c05e191a940 100644 --- a/src/Processors/QueryPlan/LimitByStep.h +++ b/src/Processors/QueryPlan/LimitByStep.h @@ -10,7 +10,7 @@ class LimitByStep : public ITransformingStep public: explicit LimitByStep( const DataStream & input_stream_, - size_t group_length_, size_t group_offset_, const Names & columns_); + size_t group_length_, size_t group_offset_, Names columns_); String getName() const override { return "LimitBy"; } @@ -19,6 +19,10 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; + void serialize(Serialization & ctx) const override; + + static std::unique_ptr deserialize(Deserialization & ctx); + private: void updateOutputStream() override { diff --git a/src/Processors/QueryPlan/LimitStep.cpp b/src/Processors/QueryPlan/LimitStep.cpp index 2e2c5ed7c1e..d25ba925f7c 100644 --- a/src/Processors/QueryPlan/LimitStep.cpp +++ b/src/Processors/QueryPlan/LimitStep.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include #include @@ -76,4 +78,47 @@ void LimitStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Reads All Data", always_read_till_end); } +void LimitStep::serialize(Serialization & ctx) const +{ + UInt8 flags = 0; + if (always_read_till_end) + flags |= 1; + if (with_ties) + flags |= 2; + + writeIntBinary(flags, ctx.out); + + writeVarUInt(limit, ctx.out); + writeVarUInt(offset, ctx.out); + + if (with_ties) + serializeSortDescription(description, ctx.out); +} + +std::unique_ptr LimitStep::deserialize(Deserialization & ctx) +{ + UInt8 flags; + readIntBinary(flags, ctx.in); + + bool always_read_till_end = bool(flags & 1); + bool with_ties = bool(flags & 2); + + UInt64 limit; + UInt64 offset; + + readVarUInt(limit, ctx.in); + readVarUInt(offset, ctx.in); + + SortDescription description; + if (with_ties) + deserializeSortDescription(description, ctx.in); + + return std::make_unique(ctx.input_streams.front(), limit, offset, always_read_till_end, with_ties, std::move(description)); +} + +void registerLimitStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("Limit", LimitStep::deserialize); +} + } diff --git a/src/Processors/QueryPlan/LimitStep.h b/src/Processors/QueryPlan/LimitStep.h index 1ae6b73cc3d..1292f949ab9 100644 --- a/src/Processors/QueryPlan/LimitStep.h +++ b/src/Processors/QueryPlan/LimitStep.h @@ -33,6 +33,10 @@ public: bool withTies() const { return with_ties; } + void serialize(Serialization & ctx) const override; + + static std::unique_ptr deserialize(Deserialization & ctx); + private: void updateOutputStream() override { diff --git a/src/Processors/QueryPlan/OffsetStep.cpp b/src/Processors/QueryPlan/OffsetStep.cpp index 4bbe81f9169..a25507686a5 100644 --- a/src/Processors/QueryPlan/OffsetStep.cpp +++ b/src/Processors/QueryPlan/OffsetStep.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include #include @@ -46,4 +48,22 @@ void OffsetStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Offset", offset); } +void OffsetStep::serialize(Serialization & ctx) const +{ + writeVarUInt(offset, ctx.out); +} + +std::unique_ptr OffsetStep::deserialize(Deserialization & ctx) +{ + UInt64 offset; + readVarUInt(offset, ctx.in); + + return std::make_unique(ctx.input_streams.front(), offset); +} + +void registerOffsetStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("Offset", OffsetStep::deserialize); +} + } diff --git a/src/Processors/QueryPlan/OffsetStep.h b/src/Processors/QueryPlan/OffsetStep.h index a32835b62a6..9c406fe6c7d 100644 --- a/src/Processors/QueryPlan/OffsetStep.h +++ b/src/Processors/QueryPlan/OffsetStep.h @@ -18,6 +18,10 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; + void serialize(Serialization & ctx) const override; + + static std::unique_ptr deserialize(Deserialization & ctx); + private: void updateOutputStream() override { diff --git a/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp b/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp index cb731376473..b1997712d25 100644 --- a/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/limitPushDown.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB::QueryPlanOptimizations @@ -63,6 +64,12 @@ size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes &) if (tryUpdateLimitForSortingSteps(child_node, limit->getLimitForSorting())) return 0; + if (auto * distinct = typeid_cast(child.get())) + { + distinct->updateLimitHint(limit->getLimitForSorting()); + return 0; + } + /// Special case for TotalsHaving. Totals may be incorrect if we push down limit. if (typeid_cast(child.get())) return 0; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 9a39df26241..96b20e9b1a3 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -2,9 +2,6 @@ #include -#include -#include - #include #include @@ -14,7 +11,6 @@ #include #include #include -#include #include #include diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 75c577af24e..12ffda337e2 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -1,9 +1,11 @@ #pragma once #include +#include #include #include #include +#include #include #include @@ -20,6 +22,7 @@ using QueryPlanStepPtr = std::unique_ptr; class QueryPipelineBuilder; using QueryPipelineBuilderPtr = std::unique_ptr; +class ReadBuffer; class WriteBuffer; class QueryPlan; @@ -30,12 +33,15 @@ class Pipe; struct QueryPlanOptimizationSettings; struct BuildQueryPipelineSettings; +class ColumnSet; namespace JSONBuilder { class IItem; using ItemPtr = std::unique_ptr; } +struct QueryPlanAndSets; + /// A tree of query steps. /// The goal of QueryPlan is to build QueryPipeline. /// QueryPlan let delay pipeline creation which is helpful for pipeline-level optimizations. @@ -54,6 +60,12 @@ public: bool isCompleted() const; /// Tree is not empty and root hasOutputStream() const DataStream & getCurrentDataStream() const; /// Checks that (isInitialized() && !isCompleted()) + void serialize(WriteBuffer & out) const; + static QueryPlanAndSets deserialize(ReadBuffer & in, const ContextPtr & context); + + static void resolveReadFromTable(QueryPlan & plan, const ContextPtr & context); + static QueryPlan resolveStorages(QueryPlanAndSets plan_and_sets, const ContextPtr & context); + void optimize(const QueryPlanOptimizationSettings & optimization_settings); QueryPipelineBuilderPtr buildQueryPipeline( @@ -125,6 +137,45 @@ private: bool concurrency_control = false; }; +class FutureSetFromSubquery; +using FutureSetFromSubqueryPtr = std::shared_ptr; + +/// This is a structure which contains a query plan and a list of StorageSet. +/// The reason is that StorageSet is specified by name, +/// and we do not want to resolve the storage name while deserializing. +/// Now, it allows to deserialize the plan without the context. +/// Potentially, it may help to get the atomic snapshot for all the storages. +/// +/// Use resolveStorages to get an ordinary plan. +struct QueryPlanAndSets +{ + struct Set + { + CityHash_v1_0_2::uint128 hash; + std::list columns; + }; + struct SetFromStorage : public Set + { + std::string storage_name; + }; + + struct SetFromTuple : public Set + { + ColumnsWithTypeAndName set_columns; + }; + + struct SetFromSubquery : public Set + { + std::unique_ptr plan; + std::list sets; + }; + + QueryPlan plan; + std::list sets_from_storage; + std::list sets_from_tuple; + std::list sets_from_subquery; +}; + std::string debugExplainStep(const IQueryPlanStep & step); } diff --git a/src/Processors/QueryPlan/QueryPlanSerializationSettings.cpp b/src/Processors/QueryPlan/QueryPlanSerializationSettings.cpp new file mode 100644 index 00000000000..9b11132e27c --- /dev/null +++ b/src/Processors/QueryPlan/QueryPlanSerializationSettings.cpp @@ -0,0 +1,8 @@ +#include + +namespace DB +{ + +IMPLEMENT_SETTINGS_TRAITS(QueryPlanSerializationSettingsTraits, PLAN_SERIALIZATION_SETTINGS); + +} diff --git a/src/Processors/QueryPlan/QueryPlanSerializationSettings.h b/src/Processors/QueryPlan/QueryPlanSerializationSettings.h new file mode 100644 index 00000000000..4e119051a51 --- /dev/null +++ b/src/Processors/QueryPlan/QueryPlanSerializationSettings.h @@ -0,0 +1,59 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +#define PLAN_SERIALIZATION_SETTINGS(M, ALIAS) \ + M(UInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size in rows for reading", 0) \ + \ + M(UInt64, max_rows_in_distinct, 0, "Maximum number of elements during execution of DISTINCT.", 0) \ + M(UInt64, max_bytes_in_distinct, 0, "Maximum total size of state (in uncompressed bytes) in memory for the execution of DISTINCT.", 0) \ + M(OverflowMode, distinct_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + \ + M(UInt64, max_rows_to_sort, 0, "If more than the specified amount of records have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ + M(UInt64, max_bytes_to_sort, 0, "If more than the specified amount of (uncompressed) bytes have to be processed for ORDER BY operation, the behavior will be determined by the 'sort_overflow_mode' which by default is - throw an exception", 0) \ + M(OverflowMode, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + \ + M(UInt64, prefer_external_sort_block_bytes, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging.", 0) \ + M(UInt64, max_bytes_before_external_sort, 0, "If memory usage during ORDER BY operation is exceeding this threshold in bytes, activate the 'external sorting' mode (spill data to disk). Recommended value is half of available system memory.", 0) \ + M(UInt64, max_bytes_before_remerge_sort, 1000000000, "In case of ORDER BY with LIMIT, when memory usage is higher than specified threshold, perform additional steps of merging blocks before final merge to keep just top LIMIT rows.", 0) \ + M(Float, remerge_sort_lowered_memory_bytes_ratio, 2., "If memory usage after remerge does not reduced by this ratio, remerge will be disabled.", 0) \ + M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ + \ + M(UInt64, aggregation_in_order_max_block_bytes, 50000000, "Maximal size of block in bytes accumulated during aggregation in order of primary key. Lower block size allows to parallelize more final merge stage of aggregation.", 0) \ + M(Bool, aggregation_in_order_memory_bound_merging, true, "Enable memory bound merging strategy when in-order is applied.", 0) \ + M(Bool, aggregation_sort_result_by_bucket_number, true, "Send intermediate aggregation result in order of bucket number.", 0) \ + \ + M(UInt64, max_rows_to_group_by, 0, "If aggregation during GROUP BY is generating more than the specified number of rows (unique GROUP BY keys), the behavior will be determined by the 'group_by_overflow_mode' which by default is - throw an exception, but can be also switched to an approximate GROUP BY mode.", 0) \ + M(OverflowModeGroupBy, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + M(UInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \ + M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ + M(UInt64, max_bytes_before_external_group_by, 0, "If memory usage during GROUP BY operation is exceeding this threshold in bytes, activate the 'external aggregation' mode (spill data to disk). Recommended value is half of available system memory.", 0) \ + M(Bool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \ + M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \ + M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \ + M(Bool, enable_software_prefetch_in_aggregation, true, "Enable use of software prefetch in aggregation", 0) \ + M(Bool, optimize_group_by_constant_keys, true, "Optimize GROUP BY when all keys in block are constant", 0) \ + M(Float, min_hit_rate_to_use_consecutive_keys_optimization, 0.5, "Minimal hit rate of a cache which is used for consecutive keys optimization in aggregation to keep it enabled", 0) \ + M(Bool, collect_hash_table_stats_during_aggregation, true, "Enable collecting hash table statistics to optimize memory allocation", 0) \ + M(UInt64, max_entries_for_hash_table_stats, 10'000, "How many entries hash table statistics collected during aggregation is allowed to have", 0) \ + M(UInt64, max_size_to_preallocate_for_aggregation, 100'000'000, "For how many elements it is allowed to preallocate space in all hash tables in total before aggregation", 0) \ + \ + M(TotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = ‘any’ are present.", IMPORTANT) \ + M(Float, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 0) \ + \ + + +DECLARE_SETTINGS_TRAITS(QueryPlanSerializationSettingsTraits, PLAN_SERIALIZATION_SETTINGS) + +struct QueryPlanSerializationSettings : public BaseSettings +{ + QueryPlanSerializationSettings() = default; +}; + +} diff --git a/src/Processors/QueryPlan/QueryPlanStepRegistry.cpp b/src/Processors/QueryPlan/QueryPlanStepRegistry.cpp new file mode 100644 index 00000000000..7918e46381c --- /dev/null +++ b/src/Processors/QueryPlan/QueryPlanStepRegistry.cpp @@ -0,0 +1,74 @@ +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_IDENTIFIER; + extern const int LOGICAL_ERROR; +} + +QueryPlanStepRegistry & QueryPlanStepRegistry::instance() +{ + static QueryPlanStepRegistry registry; + return registry; +} + +void QueryPlanStepRegistry::registerStep(const std::string & name, StepCreateFunction && create_function) +{ + if (steps.contains(name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Query plan step '{}' is already registered", name); + steps[name] = std::move(create_function); +} + +QueryPlanStepPtr QueryPlanStepRegistry::createStep( + const std::string & name, + IQueryPlanStep::Deserialization & ctx) const +{ + StepCreateFunction create_function; + { + auto it = steps.find(name); + if (it == steps.end()) + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown query plan step: {}", name); + create_function = it->second; + } + return create_function(ctx); +} + +void registerExpressionStep(QueryPlanStepRegistry & registry); +void registerUnionStep(QueryPlanStepRegistry & registry); +void registerDistinctStep(QueryPlanStepRegistry & registry); +void registerSortingStep(QueryPlanStepRegistry & registry); +void registerAggregatingStep(QueryPlanStepRegistry & registry); +void registerArrayJoinStep(QueryPlanStepRegistry & registry); +void registerLimitByStep(QueryPlanStepRegistry & registry); +void registerLimitStep(QueryPlanStepRegistry & registry); +void registerOffsetStep(QueryPlanStepRegistry & registry); +void registerFilterStep(QueryPlanStepRegistry & registry); +void registerTotalsHavingStep(QueryPlanStepRegistry & registry); + +void registerReadFromTableStep(QueryPlanStepRegistry & registry); +void registerReadFromTableFunctionStep(QueryPlanStepRegistry & registry); + +void QueryPlanStepRegistry::registerPlanSteps() +{ + QueryPlanStepRegistry & registry = QueryPlanStepRegistry::instance(); + + registerExpressionStep(registry); + registerUnionStep(registry); + registerDistinctStep(registry); + registerSortingStep(registry); + registerAggregatingStep(registry); + registerArrayJoinStep(registry); + registerLimitByStep(registry); + registerLimitStep(registry); + registerOffsetStep(registry); + registerFilterStep(registry); + registerTotalsHavingStep(registry); + + registerReadFromTableStep(registry); + registerReadFromTableFunctionStep(registry); +} + +} diff --git a/src/Processors/QueryPlan/QueryPlanStepRegistry.h b/src/Processors/QueryPlan/QueryPlanStepRegistry.h new file mode 100644 index 00000000000..fdd1232e102 --- /dev/null +++ b/src/Processors/QueryPlan/QueryPlanStepRegistry.h @@ -0,0 +1,32 @@ +#pragma once + +#include + +namespace DB +{ + +class QueryPlanStepRegistry +{ +public: + using StepCreateFunction = std::function; + + QueryPlanStepRegistry() = default; + QueryPlanStepRegistry(const QueryPlanStepRegistry &) = delete; + QueryPlanStepRegistry & operator=(const QueryPlanStepRegistry &) = delete; + + static QueryPlanStepRegistry & instance(); + + static void registerPlanSteps(); + + void registerStep(const std::string & name, StepCreateFunction && create_function); + + QueryPlanStepPtr createStep( + const std::string & name, + IQueryPlanStep::Deserialization & ctx) const; + +private: + std::unordered_map steps; + +}; + +} diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 60eadb80496..d6d7598180b 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1909,11 +1909,13 @@ bool ReadFromMergeTree::isQueryWithSampling() const if (context->getSettingsRef().parallel_replicas_count > 1 && data.supportsSampling()) return true; - const auto & select = query_info.query->as(); if (query_info.table_expression_modifiers) return query_info.table_expression_modifiers->getSampleSizeRatio() != std::nullopt; else + { + const auto & select = query_info.query->as(); return select.sampleSize() != nullptr; + } } Pipe ReadFromMergeTree::spreadMarkRanges( diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 3df46eb1987..dd595c684ce 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -209,11 +209,12 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream connections.emplace_back(std::move(try_result.entry)); String query_string = formattedAST(query); + auto stage_to_use = my_shard.query_plan ? QueryProcessingStage::QueryPlan : my_stage; my_scalars["_shard_num"] = Block{{DataTypeUInt32().createColumnConst(1, my_shard.shard_info.shard_num), std::make_shared(), "_shard_num"}}; auto remote_query_executor = std::make_shared( - std::move(connections), query_string, header, my_context, my_throttler, my_scalars, my_external_tables, my_stage); + std::move(connections), query_string, header, my_context, my_throttler, my_scalars, my_external_tables, stage_to_use); auto pipe = createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read, async_query_sending); QueryPipelineBuilder builder; @@ -285,6 +286,8 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact GetPriorityForLoadBalancing::Func priority_func = priority_func_factory->getPriorityFunc(LoadBalancing::ROUND_ROBIN, 0, shard.shard_info.pool->getPoolSize()); + auto stage_to_use = shard.query_plan ? QueryProcessingStage::QueryPlan : stage; + auto remote_query_executor = std::make_shared( shard.shard_info.pool, query_string, @@ -293,7 +296,8 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact throttler, scalars, external_tables, - stage, + stage_to_use, + shard.query_plan, std::nullopt, priority_func); remote_query_executor->setLogger(log); @@ -309,10 +313,12 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFact } else { + const String query_string = formattedAST(shard.query); + auto stage_to_use = shard.query_plan ? QueryProcessingStage::QueryPlan : stage; auto remote_query_executor = std::make_shared( - shard.shard_info.pool, query_string, shard.header, context, throttler, scalars, external_tables, stage); + shard.shard_info.pool, query_string, shard.header, context, throttler, scalars, external_tables, stage_to_use, shard.query_plan); remote_query_executor->setLogger(log); if (context->canUseTaskBasedParallelReplicas()) diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 596d08845e1..f2dc237f3a1 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -348,6 +348,9 @@ namespace /// Whether we should push limit down to scan. bool shouldPushdownLimit(SelectQueryInfo & query_info, UInt64 limit_length) { + if (!query_info.query) + return false; + const auto & query = query_info.query->as(); /// Just ignore some minor cases, such as: /// select * from system.numbers order by number asc limit 10 @@ -409,7 +412,7 @@ ReadFromSystemNumbersStep::ReadFromSystemNumbersStep( , key_expression{KeyDescription::parse(column_names[0], storage_snapshot->metadata->columns, context).expression} , max_block_size{max_block_size_} , num_streams{num_streams_} - , limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as(), context)) + , limit_length_and_offset(query_info.query ? InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as(), context) : std::make_pair(UInt64(0), UInt64(0))) , should_pushdown_limit(shouldPushdownLimit(query_info, limit_length_and_offset.first)) , query_info_limit(query_info.trivial_limit) , storage_limits(query_info.storage_limits) diff --git a/src/Processors/QueryPlan/ReadFromTableFunctionStep.cpp b/src/Processors/QueryPlan/ReadFromTableFunctionStep.cpp new file mode 100644 index 00000000000..510a965ed47 --- /dev/null +++ b/src/Processors/QueryPlan/ReadFromTableFunctionStep.cpp @@ -0,0 +1,107 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +ReadFromTableFunctionStep::ReadFromTableFunctionStep( + Block output_header, + std::string serialized_ast_, + TableExpressionModifiers table_expression_modifiers_) + : ISourceStep(DataStream{.header = std::move(output_header)}) + , serialized_ast(std::move(serialized_ast_)) + , table_expression_modifiers(std::move(table_expression_modifiers_)) +{ +} + +void ReadFromTableFunctionStep::initializePipeline(QueryPipelineBuilder &, const BuildQueryPipelineSettings &) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "initializePipeline is not implementad for ReadFromTableFunctionStep"); +} + +static void serializeRational(TableExpressionModifiers::Rational val, WriteBuffer & out) +{ + writeIntBinary(val.numerator, out); + writeIntBinary(val.denominator, out); +} + +static TableExpressionModifiers::Rational deserializeRational(ReadBuffer & in) +{ + TableExpressionModifiers::Rational val; + readIntBinary(val.numerator, in); + readIntBinary(val.denominator, in); + return val; +} + +enum class TableFunctionSerializationKind : UInt8 +{ + AST = 0, +}; + +void ReadFromTableFunctionStep::serialize(Serialization & ctx) const +{ + writeIntBinary(TableFunctionSerializationKind::AST, ctx.out); + + writeStringBinary(serialized_ast, ctx.out); + + UInt8 flags = 0; + if (table_expression_modifiers.hasFinal()) + flags |= 1; + if (table_expression_modifiers.hasSampleSizeRatio()) + flags |= 2; + if (table_expression_modifiers.hasSampleOffsetRatio()) + flags |= 4; + + writeIntBinary(flags, ctx.out); + if (table_expression_modifiers.hasSampleSizeRatio()) + serializeRational(*table_expression_modifiers.getSampleSizeRatio(), ctx.out); + + if (table_expression_modifiers.hasSampleOffsetRatio()) + serializeRational(*table_expression_modifiers.getSampleOffsetRatio(), ctx.out); +} + +std::unique_ptr ReadFromTableFunctionStep::deserialize(Deserialization & ctx) +{ + UInt8 kind; + readIntBinary(kind, ctx.in); + + if (kind != UInt8(TableFunctionSerializationKind::AST)) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Serialization kind {} is not implemented for ReadFromTableFunctionStep", int(kind)); + + String serialized_ast; + readStringBinary(serialized_ast, ctx.in); + + UInt8 flags = 0; + readIntBinary(flags, ctx.in); + + bool has_final = false; + std::optional sample_size_ratio; + std::optional sample_offset_ratio; + + if (flags & 1) + has_final = true; + + if (flags & 2) + sample_size_ratio = deserializeRational(ctx.in); + + if (flags & 4) + sample_offset_ratio = deserializeRational(ctx.in); + + TableExpressionModifiers table_expression_modifiers(has_final, sample_size_ratio, sample_offset_ratio); + return std::make_unique(ctx.output_stream->header, std::move(serialized_ast), table_expression_modifiers); +} + +void registerReadFromTableFunctionStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("ReadFromTableFunction", &ReadFromTableFunctionStep::deserialize); +} + +} diff --git a/src/Processors/QueryPlan/ReadFromTableFunctionStep.h b/src/Processors/QueryPlan/ReadFromTableFunctionStep.h new file mode 100644 index 00000000000..056a1472fd0 --- /dev/null +++ b/src/Processors/QueryPlan/ReadFromTableFunctionStep.h @@ -0,0 +1,28 @@ +#pragma once +#include +#include + +namespace DB +{ + +class ReadFromTableFunctionStep : public ISourceStep +{ +public: + ReadFromTableFunctionStep(Block output_header, std::string serialized_ast_, TableExpressionModifiers table_expression_modifiers_); + + String getName() const override { return "ReadFromTableFunction"; } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + void serialize(Serialization & ctx) const override; + static std::unique_ptr deserialize(Deserialization & ctx); + + const std::string & getSerializedAST() const { return serialized_ast; } + TableExpressionModifiers getTableExpressionModifiers() const { return table_expression_modifiers; } + +private: + std::string serialized_ast; + TableExpressionModifiers table_expression_modifiers; +}; + +} diff --git a/src/Processors/QueryPlan/ReadFromTableStep.cpp b/src/Processors/QueryPlan/ReadFromTableStep.cpp new file mode 100644 index 00000000000..c7ca56efc99 --- /dev/null +++ b/src/Processors/QueryPlan/ReadFromTableStep.cpp @@ -0,0 +1,94 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +ReadFromTableStep::ReadFromTableStep( + Block output_header, + String table_name_, + TableExpressionModifiers table_expression_modifiers_) + : ISourceStep(DataStream{.header = std::move(output_header)}) + , table_name(std::move(table_name_)) + , table_expression_modifiers(std::move(table_expression_modifiers_)) +{ +} + +void ReadFromTableStep::initializePipeline(QueryPipelineBuilder &, const BuildQueryPipelineSettings &) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "initializePipeline is not implementad for ReadFromTableStep"); +} + +static void serializeRational(TableExpressionModifiers::Rational val, WriteBuffer & out) +{ + writeIntBinary(val.numerator, out); + writeIntBinary(val.denominator, out); +} + +static TableExpressionModifiers::Rational deserializeRational(ReadBuffer & in) +{ + TableExpressionModifiers::Rational val; + readIntBinary(val.numerator, in); + readIntBinary(val.denominator, in); + return val; +} + +void ReadFromTableStep::serialize(Serialization & ctx) const +{ + writeStringBinary(table_name, ctx.out); + + UInt8 flags = 0; + if (table_expression_modifiers.hasFinal()) + flags |= 1; + if (table_expression_modifiers.hasSampleSizeRatio()) + flags |= 2; + if (table_expression_modifiers.hasSampleOffsetRatio()) + flags |= 4; + + writeIntBinary(flags, ctx.out); + if (table_expression_modifiers.hasSampleSizeRatio()) + serializeRational(*table_expression_modifiers.getSampleSizeRatio(), ctx.out); + + if (table_expression_modifiers.hasSampleOffsetRatio()) + serializeRational(*table_expression_modifiers.getSampleOffsetRatio(), ctx.out); +} + +std::unique_ptr ReadFromTableStep::deserialize(Deserialization & ctx) +{ + String table_name; + readStringBinary(table_name, ctx.in); + + UInt8 flags = 0; + readIntBinary(flags, ctx.in); + + bool has_final = false; + std::optional sample_size_ratio; + std::optional sample_offset_ratio; + + if (flags & 1) + has_final = true; + + if (flags & 2) + sample_size_ratio = deserializeRational(ctx.in); + + if (flags & 4) + sample_offset_ratio = deserializeRational(ctx.in); + + TableExpressionModifiers table_expression_modifiers(has_final, sample_size_ratio, sample_offset_ratio); + return std::make_unique(ctx.output_stream->header, table_name, table_expression_modifiers); +} + +void registerReadFromTableStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("ReadFromTable", &ReadFromTableStep::deserialize); +} + +} diff --git a/src/Processors/QueryPlan/ReadFromTableStep.h b/src/Processors/QueryPlan/ReadFromTableStep.h new file mode 100644 index 00000000000..4e5a48a3ae2 --- /dev/null +++ b/src/Processors/QueryPlan/ReadFromTableStep.h @@ -0,0 +1,28 @@ +#pragma once +#include +#include + +namespace DB +{ + +class ReadFromTableStep : public ISourceStep +{ +public: + ReadFromTableStep(Block output_header, String table_name_, TableExpressionModifiers table_expression_modifiers_); + + String getName() const override { return "ReadFromTable"; } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + void serialize(Serialization & ctx) const override; + static std::unique_ptr deserialize(Deserialization & ctx); + + const String & getTable() const { return table_name; } + TableExpressionModifiers getTableExpressionModifiers() const { return table_expression_modifiers; } + +private: + String table_name; + TableExpressionModifiers table_expression_modifiers; +}; + +} diff --git a/src/Processors/QueryPlan/Serialization.cpp b/src/Processors/QueryPlan/Serialization.cpp new file mode 100644 index 00000000000..22457307493 --- /dev/null +++ b/src/Processors/QueryPlan/Serialization.cpp @@ -0,0 +1,632 @@ +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; + extern const int UNKNOWN_TABLE; +} + +static void serializeHeader(const Block & header, WriteBuffer & out) +{ + /// Write only names and types. + /// Constants should be filled by step. + + writeVarUInt(header.columns(), out); + for (const auto & column : header) + { + writeStringBinary(column.name, out); + encodeDataType(column.type, out); + } +} + +static Block deserializeHeader(ReadBuffer & in) +{ + UInt64 num_columns; + readVarUInt(num_columns, in); + + ColumnsWithTypeAndName columns(num_columns); + + for (auto & column : columns) + { + readStringBinary(column.name, in); + column.type = decodeDataType(in); + } + + /// Fill columns in header. Some steps expect them to be not empty. + for (auto & column : columns) + column.column = column.type->createColumn(); + + return Block(std::move(columns)); +} + +enum class SetSerializationKind : UInt8 +{ + StorageSet = 1, + TupleValues = 2, + SubqueryPlan = 3, +}; + +static void serializeSets(SerializedSetsRegistry & registry, WriteBuffer & out) +{ + writeVarUInt(registry.sets.size(), out); + for (const auto & [hash, set] : registry.sets) + { + writeBinary(hash, out); + + if (auto * from_storage = typeid_cast(set.get())) + { + writeIntBinary(SetSerializationKind::StorageSet, out); + const auto & storage_id = from_storage->getStorageID(); + if (!storage_id) + throw Exception(ErrorCodes::LOGICAL_ERROR, "FutureSetFromStorage without storage id"); + + auto storage_name = storage_id->getFullTableName(); + writeStringBinary(storage_name, out); + } + else if (auto * from_tuple = typeid_cast(set.get())) + { + writeIntBinary(SetSerializationKind::TupleValues, out); + + auto types = from_tuple->getTypes(); + auto columns = from_tuple->getKeyColumns(); + + if (columns.size() != types.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid number of columns for Set. Expected {} got {}", + columns.size(), types.size()); + + UInt64 num_columns = columns.size(); + UInt64 num_rows = num_columns > 0 ? columns.front()->size() : 0; + + writeVarUInt(num_columns, out); + writeVarUInt(num_rows, out); + + for (size_t col = 0; col < num_columns; ++col) + { + if (columns[col]->size() != num_rows) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid number of rows in column of Set. Expected {} got {}", + num_rows, columns[col]->size()); + + encodeDataType(types[col], out); + auto serialization = types[col]->getSerialization(ISerialization::Kind::DEFAULT); + serialization->serializeBinaryBulk(*columns[col], out, 0, num_rows); + } + } + else if (auto * from_subquery = typeid_cast(set.get())) + { + writeIntBinary(SetSerializationKind::SubqueryPlan, out); + const auto * plan = from_subquery->getQueryPlan(); + if (!plan) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot serialize FutureSetFromSubquery with no query plan"); + + plan->serialize(out); + } + else + { + const auto & set_ref = *set; + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown FutureSet type {}", typeid(set_ref).name()); + } + } +} + +QueryPlanAndSets deserializeSets(QueryPlan plan, DeserializedSetsRegistry & registry, ReadBuffer & in, const ContextPtr & context) +{ + UInt64 num_sets; + readVarUInt(num_sets, in); + + QueryPlanAndSets res; + res.plan = std::move(plan); + + for (size_t i = 0; i < num_sets; ++i) + { + PreparedSets::Hash hash; + readBinary(hash, in); + + auto it = registry.sets.find(hash); + if (it == registry.sets.end()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Serialized set {}_{} is not registered", hash.low64, hash.high64); + + auto & columns = it->second; + if (columns.empty()) + throw Exception(ErrorCodes::INCORRECT_DATA, "Serialized set {}_{} is serialized twice", hash.low64, hash.high64); + + UInt8 kind; + readVarUInt(kind, in); + if (kind == UInt8(SetSerializationKind::StorageSet)) + { + String storage_name; + readStringBinary(storage_name, in); + res.sets_from_storage.emplace_back(QueryPlanAndSets::SetFromStorage{{hash, std::move(columns)}, std::move(storage_name)}); + } + else if (kind == UInt8(SetSerializationKind::TupleValues)) + { + UInt64 num_columns; + UInt64 num_rows; + readVarUInt(num_columns, in); + readVarUInt(num_rows, in); + + ColumnsWithTypeAndName set_columns; + set_columns.reserve(num_columns); + + for (size_t col = 0; col < num_columns; ++col) + { + auto type = decodeDataType(in); + auto serialization = type->getSerialization(ISerialization::Kind::DEFAULT); + auto column = type->createColumn(); + serialization->deserializeBinaryBulk(*column, in, num_rows, 0); + + set_columns.emplace_back(std::move(column), std::move(type), String{}); + } + + res.sets_from_tuple.emplace_back(QueryPlanAndSets::SetFromTuple{{hash, std::move(columns)}, std::move(set_columns)}); + } + else if (kind == UInt8(SetSerializationKind::SubqueryPlan)) + { + auto plan_for_set = QueryPlan::deserialize(in, context); + + res.sets_from_subquery.emplace_back(QueryPlanAndSets::SetFromSubquery{ + {hash, std::move(columns)}, + std::make_unique(std::move(plan_for_set.plan)), + std::move(plan_for_set.sets_from_subquery)}); + + res.sets_from_storage.splice(res.sets_from_storage.end(), std::move(plan_for_set.sets_from_storage)); + res.sets_from_tuple.splice(res.sets_from_tuple.end(), std::move(plan_for_set.sets_from_tuple)); + } + else + throw Exception(ErrorCodes::INCORRECT_DATA, "Serialized set {}_{} has unknown kind {}", + hash.low64, hash.high64, int(kind)); + } + + return res; +} + +void QueryPlan::serialize(WriteBuffer & out) const +{ + checkInitialized(); + + SerializedSetsRegistry registry; + + struct Frame + { + Node * node = {}; + size_t next_child = 0; + }; + + std::stack stack; + stack.push(Frame{.node = root}); + while (!stack.empty()) + { + auto & frame = stack.top(); + auto * node = frame.node; + + if (typeid_cast(node->step.get())) + { + frame.node = node->children.front(); + continue; + } + + if (frame.next_child == 0) + { + writeVarUInt(node->children.size(), out); + } + + if (frame.next_child < node->children.size()) + { + stack.push(Frame{.node = node->children[frame.next_child]}); + ++frame.next_child; + continue; + } + + stack.pop(); + + writeStringBinary(node->step->getSerializationName(), out); + writeStringBinary(node->step->getStepDescription(), out); + + if (node->step->hasOutputStream()) + serializeHeader(node->step->getOutputStream().header, out); + else + serializeHeader({}, out); + + QueryPlanSerializationSettings settings; + node->step->serializeSettings(settings); + + settings.writeChangedBinary(out); + + IQueryPlanStep::Serialization ctx{out, registry}; + node->step->serialize(ctx); + } + + serializeSets(registry, out); +} + +QueryPlanAndSets QueryPlan::deserialize(ReadBuffer & in, const ContextPtr & context) +{ + QueryPlanStepRegistry & step_registry = QueryPlanStepRegistry::instance(); + + DeserializedSetsRegistry sets_registry; + + using NodePtr = Node *; + struct Frame + { + NodePtr & to_fill; + size_t next_child = 0; + std::vector children = {}; + }; + + std::stack stack; + + QueryPlan plan; + stack.push(Frame{.to_fill = plan.root}); + + while (!stack.empty()) + { + auto & frame = stack.top(); + if (frame.next_child == 0) + { + UInt64 num_children; + readVarUInt(num_children, in); + frame.children.resize(num_children); + } + + if (frame.next_child < frame.children.size()) + { + stack.push(Frame{.to_fill = frame.children[frame.next_child]}); + ++frame.next_child; + continue; + } + + std::string step_name; + std::string step_description; + readStringBinary(step_name, in); + readStringBinary(step_description, in); + + DataStream output_stream; + output_stream.header = deserializeHeader(in); + + QueryPlanSerializationSettings settings; + settings.readBinary(in); + + DataStreams input_streams; + input_streams.reserve(frame.children.size()); + for (const auto & child : frame.children) + input_streams.push_back(child->step->getOutputStream()); + + IQueryPlanStep::Deserialization ctx{in, sets_registry, context, input_streams, &output_stream, settings}; + auto step = step_registry.createStep(step_name, ctx); + + if (step->hasOutputStream()) + { + assertCompatibleHeader(step->getOutputStream().header, output_stream.header, + fmt::format("deserialization of query plan {} step", step_name)); + } + else if (output_stream.header.columns()) + throw Exception(ErrorCodes::INCORRECT_DATA, + "Deserialized step {} has no output stream, but deserialized header is not empty : {}", + step_name, output_stream.header.dumpStructure()); + + auto & node = plan.nodes.emplace_back(std::move(step), std::move(frame.children)); + frame.to_fill = &node; + + stack.pop(); + } + + return deserializeSets(std::move(plan), sets_registry, in, context); +} + +static std::shared_ptr resolveTable(const Identifier & identifier, const ContextPtr & context) +{ + auto table_node_ptr = IdentifierResolver::tryResolveTableIdentifierFromDatabaseCatalog(identifier, context); + if (!table_node_ptr) + throw Exception(ErrorCodes::UNKNOWN_TABLE, "Unknown table {}", identifier.getFullName()); + + return table_node_ptr; +} + +static QueryTreeNodePtr resolveTableFunction(const ASTPtr & table_function, const ContextPtr & context) +{ + QueryTreeNodePtr query_tree_node = buildTableFunctionQueryTree(table_function, context); + + bool only_analyze = false; + QueryAnalyzer analyzer(only_analyze); + analyzer.resolve(query_tree_node, nullptr, context); + + return query_tree_node; +} + +static void makeSetsFromStorage(std::list sets, const ContextPtr & context) +{ + for (auto & set : sets) + { + Identifier identifier(set.storage_name); + auto table_node = resolveTable(identifier, context); + const auto * storage_set = typeid_cast(table_node->getStorage().get()); + if (!storage_set) + throw Exception(ErrorCodes::INCORRECT_DATA, "Table {} is not a StorageSet", set.storage_name); + + auto future_set = std::make_shared(set.hash, storage_set->getSet(), table_node->getStorageID()); + for (auto * column : set.columns) + column->setData(future_set); + } +} + +static void makeSetsFromTuple(std::list sets, const ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + for (auto & set : sets) + { + SizeLimits size_limits = PreparedSets::getSizeLimitsForSet(settings); + bool transform_null_in = settings.transform_null_in; + + auto future_set = std::make_shared(set.hash, std::move(set.set_columns), transform_null_in, size_limits); + for (auto * column : set.columns) + column->setData(future_set); + } +} + +static void makeSetsFromSubqueries(QueryPlan & plan, std::list sets_from_subqueries, const ContextPtr & context) +{ + if (sets_from_subqueries.empty()) + return; + + const auto & settings = context->getSettingsRef(); + + PreparedSets::Subqueries subqueries; + subqueries.reserve(sets_from_subqueries.size()); + for (auto & set : sets_from_subqueries) + { + QueryPlan::resolveReadFromTable(*set.plan, context); + makeSetsFromSubqueries(*set.plan, std::move(set.sets), context); + + SizeLimits size_limits = PreparedSets::getSizeLimitsForSet(settings); + bool transform_null_in = settings.transform_null_in; + size_t max_size_for_index = settings.use_index_for_in_with_subqueries_max_values; + + auto future_set = std::make_shared( + set.hash, std::move(set.plan), nullptr, nullptr, + transform_null_in, size_limits, max_size_for_index); + + for (auto * column : set.columns) + column->setData(future_set); + + subqueries.push_back(std::move(future_set)); + } + + auto step = std::make_unique( + plan.getCurrentDataStream(), + std::move(subqueries), + context); + + plan.addStep(std::move(step)); +} + +static ASTPtr makeASTForReadingColumns(const Names & names, ASTPtr table_expression) +{ + auto select = std::make_shared(); + auto columns = std::make_shared(); + for (const auto & name : names) + columns->children.push_back(std::make_shared(name)); + + auto tables = std::make_shared(); + auto table_element = std::make_shared(); + table_element->children.push_back(table_expression); + table_element->table_expression = std::move(table_expression); + tables->children.push_back(std::move(table_element)); + + select->setExpression(ASTSelectQuery::Expression::SELECT, std::move(columns)); + select->setExpression(ASTSelectQuery::Expression::TABLES, std::move(tables)); + + return select; +} + +static ASTPtr wrapWithUnion(ASTPtr select) +{ + auto select_with_union = std::make_shared(); + auto selects = std::make_shared(); + selects->children.push_back(select); + select_with_union->list_of_selects = selects; + select_with_union->children.push_back(select_with_union->list_of_selects); + + return select_with_union; +} + +static QueryPlanResourceHolder replaceReadingFromTable(QueryPlan::Node & node, QueryPlan::Nodes & nodes, const ContextPtr & context) +{ + const auto * reading_from_table = typeid_cast(node.step.get()); + const auto * reading_from_table_function = typeid_cast(node.step.get()); + if (!reading_from_table && !reading_from_table_function) + return {}; + + const auto & header = node.step->getOutputStream().header; + auto column_names = header.getNames(); + + StoragePtr storage; + StorageSnapshotPtr snapshot; + SelectQueryInfo select_query_info; + ASTPtr table_function_ast; + + if (reading_from_table) + { + Identifier identifier(reading_from_table->getTable()); + auto table_node = resolveTable(identifier, context); + + storage = table_node->getStorage(); + snapshot = table_node->getStorageSnapshot(); + select_query_info.table_expression_modifiers = reading_from_table->getTableExpressionModifiers(); + } + else + { + auto serialized_ast = reading_from_table_function->getSerializedAST(); + ParserFunction parser(false, true); + const auto & settings = context->getSettingsRef(); + table_function_ast = parseQuery( + parser, + serialized_ast, + settings.max_query_size, + settings.max_parser_depth, + DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); + + auto query_tree_node = resolveTableFunction(table_function_ast, context); + if (auto * table_function_node = query_tree_node->as()) + { + storage = table_function_node->getStorage(); + snapshot = table_function_node->getStorageSnapshot(); + } + else if (auto * table_node = query_tree_node->as()) + { + storage = table_node->getStorage(); + snapshot = table_node->getStorageSnapshot(); + } + else + throw Exception(ErrorCodes::INCORRECT_DATA, + "Unexpected query tree node type {}\n{}", + query_tree_node->getNodeTypeName(), + query_tree_node->dumpTree()); + + select_query_info.table_expression_modifiers = reading_from_table_function->getTableExpressionModifiers(); + } + + ASTPtr query; + bool is_storage_merge = typeid_cast(storage.get()); + if (storage->isRemote() || is_storage_merge) + { + auto table_expression = std::make_shared(); + if (table_function_ast) + { + table_expression->children.push_back(table_function_ast); + table_expression->table_function = std::move(table_function_ast); + } + else + { + const auto & table_id = storage->getStorageID(); + auto table_identifier = std::make_shared(table_id.database_name, table_id.table_name); + table_expression->children.push_back(table_identifier); + table_identifier->uuid = table_id.uuid; + table_expression->database_and_table_name = std::move(table_identifier); + } + + query = makeASTForReadingColumns(column_names, std::move(table_expression)); + // std::cerr << query->dumpTree() << std::endl; + } + + QueryPlan reading_plan; + if (storage->isRemote() || is_storage_merge) + { + SelectQueryOptions options(QueryProcessingStage::FetchColumns); + options.ignore_rename_columns = true; + InterpreterSelectQueryAnalyzer interpreter(wrapWithUnion(std::move(query)), context, options); + reading_plan = std::move(interpreter).extractQueryPlan(); + } + else + { + SelectQueryOptions options(QueryProcessingStage::FetchColumns); + auto storage_limits = std::make_shared(); + storage_limits->emplace_back(buildStorageLimits(*context, options)); + select_query_info.storage_limits = std::move(storage_limits); + select_query_info.query = std::move(query); + + storage->read( + reading_plan, + column_names, + snapshot, + select_query_info, + context, + QueryProcessingStage::FetchColumns, + context->getSettingsRef().max_block_size, + context->getSettingsRef().max_threads + ); + } + + if (!reading_plan.isInitialized()) + { + /// Create step which reads from empty source if storage has no data. + auto source_header = snapshot->getSampleBlockForColumns(column_names); + Pipe pipe(std::make_shared(source_header)); + auto read_from_pipe = std::make_unique(std::move(pipe)); + read_from_pipe->setStepDescription("Read from NullSource"); + reading_plan.addStep(std::move(read_from_pipe)); + } + + auto converting_actions = ActionsDAG::makeConvertingActions( + reading_plan.getCurrentDataStream().header.getColumnsWithTypeAndName(), + header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + node.step = std::make_unique(reading_plan.getCurrentDataStream(), std::move(converting_actions)); + node.children = {reading_plan.getRootNode()}; + + auto nodes_and_resource = QueryPlan::detachNodesAndResources(std::move(reading_plan)); + + nodes.splice(nodes.end(), std::move(nodes_and_resource.first)); + return std::move(nodes_and_resource.second); +} + +void QueryPlan::resolveReadFromTable(QueryPlan & plan, const ContextPtr & context) +{ + std::stack stack; + stack.push(plan.getRootNode()); + while (!stack.empty()) + { + auto * node = stack.top(); + stack.pop(); + + for (auto * child : node->children) + stack.push(child); + + if (node->children.empty()) + plan.addResources(replaceReadingFromTable(*node, plan.nodes, context)); + } +} + +QueryPlan QueryPlan::resolveStorages(QueryPlanAndSets plan_and_sets, const ContextPtr & context) +{ + auto & plan = plan_and_sets.plan; + + resolveReadFromTable(plan, context); + + makeSetsFromStorage(std::move(plan_and_sets.sets_from_storage), context); + makeSetsFromTuple(std::move(plan_and_sets.sets_from_tuple), context); + makeSetsFromSubqueries(plan, std::move(plan_and_sets.sets_from_subquery), context); + + return std::move(plan); +} + +} diff --git a/src/Processors/QueryPlan/Serialization.h b/src/Processors/QueryPlan/Serialization.h new file mode 100644 index 00000000000..f03d47a152c --- /dev/null +++ b/src/Processors/QueryPlan/Serialization.h @@ -0,0 +1,30 @@ +#pragma once +#include +#include + +namespace DB +{ + +struct SerializedSetsRegistry; +struct DeserializedSetsRegistry; + +struct IQueryPlanStep::Serialization +{ + WriteBuffer & out; + SerializedSetsRegistry & registry; +}; + +struct SerializedSetsRegistry; + +struct IQueryPlanStep::Deserialization +{ + ReadBuffer & in; + DeserializedSetsRegistry & registry; + const ContextPtr & context; + + const DataStreams & input_streams; + const DataStream * output_stream; + const QueryPlanSerializationSettings & settings; +}; + +} diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index 48fad9f5fdb..ea6d9aa69ed 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -2,6 +2,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -27,6 +30,8 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; + extern const int INCORRECT_DATA; } SortingStep::Settings::Settings(const Context & context) @@ -48,6 +53,33 @@ SortingStep::Settings::Settings(size_t max_block_size_) max_block_size = max_block_size_; } +SortingStep::Settings::Settings(const QueryPlanSerializationSettings & settings) +{ + max_block_size = settings.max_block_size; + size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; + remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; + max_bytes_before_external_sort = settings.max_bytes_before_external_sort; + tmp_data = Context::getGlobalContextInstance()->getTempDataOnDisk(); + min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + max_block_bytes = settings.prefer_external_sort_block_bytes; + read_in_order_use_buffering = false; //settings.read_in_order_use_buffering; +} + +void SortingStep::Settings::updatePlanSettings(QueryPlanSerializationSettings & settings) const +{ + settings.max_block_size = max_block_size; + settings.max_rows_to_sort = size_limits.max_rows; + settings.max_bytes_to_sort = size_limits.max_bytes; + settings.sort_overflow_mode = size_limits.overflow_mode; + + settings.max_bytes_before_remerge_sort = max_bytes_before_remerge; + settings.remerge_sort_lowered_memory_bytes_ratio = remerge_lowered_memory_bytes_ratio; + settings.max_bytes_before_external_sort = max_bytes_before_external_sort; + settings.min_free_disk_space_for_temporary_data = min_free_disk_space; + settings.prefer_external_sort_block_bytes = max_block_bytes; +} + static ITransformingStep::Traits getTraits(size_t limit) { return ITransformingStep::Traits @@ -465,4 +497,52 @@ void SortingStep::describeActions(JSONBuilder::JSONMap & map) const map.add("Limit", limit); } +void SortingStep::serializeSettings(QueryPlanSerializationSettings & settings) const +{ + sort_settings.updatePlanSettings(settings); +} + +void SortingStep::serialize(Serialization & ctx) const +{ + if (type != Type::Full) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Serialization of SortingStep is implemented only for Full sorting"); + + /// Do not serialize type here; Later we can use different names if needed.\ + + /// Do not serialize limit for now; it is expected to be pushed down from plan optimization. + + serializeSortDescription(result_description, ctx.out); + + /// Later + if (!partition_by_description.empty()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Serialization of partitioned sorting is not implemented for SortingStep"); + + writeVarUInt(partition_by_description.size(), ctx.out); +} + +std::unique_ptr SortingStep::deserialize(Deserialization & ctx) +{ + if (ctx.input_streams.size() != 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "SortingStep must have one input stream"); + + SortingStep::Settings sort_settings(ctx.settings); + + SortDescription result_description; + deserializeSortDescription(result_description, ctx.in); + + UInt64 partition_desc_size; + readVarUInt(partition_desc_size, ctx.in); + + if (partition_desc_size) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Deserialization of partitioned sorting is not implemented for SortingStep"); + + return std::make_unique( + ctx.input_streams.front(), std::move(result_description), 0, std::move(sort_settings), true); +} + +void registerSortingStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("Sorting", SortingStep::deserialize); +} + } diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index b4a49394a13..46ce1e7df78 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -23,7 +23,7 @@ public: size_t max_block_size; SizeLimits size_limits; size_t max_bytes_before_remerge = 0; - double remerge_lowered_memory_bytes_ratio = 0; + float remerge_lowered_memory_bytes_ratio = 0; size_t max_bytes_before_external_sort = 0; TemporaryDataOnDiskScopePtr tmp_data = nullptr; size_t min_free_disk_space = 0; @@ -32,6 +32,9 @@ public: explicit Settings(const Context & context); explicit Settings(size_t max_block_size_); + explicit Settings(const QueryPlanSerializationSettings & settings); + + void updatePlanSettings(QueryPlanSerializationSettings & settings) const; }; /// Full @@ -93,6 +96,11 @@ public: UInt64 limit_, bool skip_partial_sort = false); + void serializeSettings(QueryPlanSerializationSettings & settings) const override; + void serialize(Serialization & ctx) const override; + + static std::unique_ptr deserialize(Deserialization & ctx); + private: void scatterByPartitionIfNeeded(QueryPipelineBuilder& pipeline); void updateOutputStream() override; diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 2554053064f..6ef17e73ae6 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -1,4 +1,7 @@ #include +#include +#include +#include #include #include #include @@ -10,6 +13,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + static ITransformingStep::Traits getTraits(bool has_filter) { return ITransformingStep::Traits @@ -33,7 +41,7 @@ TotalsHavingStep::TotalsHavingStep( const std::string & filter_column_, bool remove_filter_, TotalsMode totals_mode_, - double auto_include_threshold_, + float auto_include_threshold_, bool final_) : ITransformingStep( input_stream_, @@ -143,5 +151,75 @@ void TotalsHavingStep::updateOutputStream() getDataStreamTraits()); } +void TotalsHavingStep::serializeSettings(QueryPlanSerializationSettings & settings) const +{ + settings.totals_mode = totals_mode; + settings.totals_auto_threshold = auto_include_threshold; +} + +void TotalsHavingStep::serialize(Serialization & ctx) const +{ + UInt8 flags = 0; + if (final) + flags |= 1; + if (overflow_row) + flags |= 2; + if (actions_dag) + flags |= 4; + if (actions_dag && remove_filter) + flags |= 8; + + writeIntBinary(flags, ctx.out); + + serializeAggregateDescriptions(aggregates, ctx.out); + + if (actions_dag) + { + writeStringBinary(filter_column_name, ctx.out); + actions_dag->serialize(ctx.out, ctx.registry); + } +} + +std::unique_ptr TotalsHavingStep::deserialize(Deserialization & ctx) +{ + if (ctx.input_streams.size() != 1) + throw Exception(ErrorCodes::INCORRECT_DATA, "TotalsHaving must have one input stream"); + + UInt8 flags; + readIntBinary(flags, ctx.in); + + bool final = bool(flags & 1); + bool overflow_row = bool(flags & 2); + bool has_actions_dag = bool(flags & 4); + bool remove_filter_column = bool(flags & 8); + + AggregateDescriptions aggregates; + deserializeAggregateDescriptions(aggregates, ctx.in, ctx.input_streams.front().header); + + std::optional actions_dag; + String filter_column_name; + if (has_actions_dag) + { + readStringBinary(filter_column_name, ctx.in); + + actions_dag = ActionsDAG::deserialize(ctx.in, ctx.registry, ctx.context); + } + + return std::make_unique( + ctx.input_streams.front(), + std::move(aggregates), + overflow_row, + std::move(actions_dag), + std::move(filter_column_name), + remove_filter_column, + ctx.settings.totals_mode, + ctx.settings.totals_auto_threshold, + final); +} + +void registerTotalsHavingStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("TotalsHaving", TotalsHavingStep::deserialize); +} } diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index 4b414d41c57..96a321cbf44 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -20,7 +20,7 @@ public: const std::string & filter_column_, bool remove_filter_, TotalsMode totals_mode_, - double auto_include_threshold_, + float auto_include_threshold_, bool final_); String getName() const override { return "TotalsHaving"; } @@ -32,6 +32,11 @@ public: const ActionsDAG * getActions() const { return actions_dag ? &*actions_dag : nullptr; } + void serializeSettings(QueryPlanSerializationSettings & settings) const override; + void serialize(Serialization & ctx) const override; + + static std::unique_ptr deserialize(Deserialization & ctx); + private: void updateOutputStream() override; @@ -42,7 +47,7 @@ private: String filter_column_name; bool remove_filter; TotalsMode totals_mode; - double auto_include_threshold; + float auto_include_threshold; bool final; }; diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index dde12271de1..976bd427d4c 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include #include #include @@ -57,7 +59,7 @@ void UnionStep::updateOutputStream() } } -QueryPipelineBuilderPtr UnionStep::updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) +QueryPipelineBuilderPtr UnionStep::updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings & settings) { auto pipeline = std::make_unique(); @@ -69,6 +71,8 @@ QueryPipelineBuilderPtr UnionStep::updatePipeline(QueryPipelineBuilders pipeline return pipeline; } + size_t new_max_threads = max_threads ? max_threads : settings.max_threads; + for (auto & cur_pipeline : pipelines) { #if !defined(NDEBUG) @@ -95,7 +99,7 @@ QueryPipelineBuilderPtr UnionStep::updatePipeline(QueryPipelineBuilders pipeline } } - *pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines), max_threads, &processors); + *pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines), new_max_threads, &processors); return pipeline; } @@ -104,4 +108,19 @@ void UnionStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } +void UnionStep::serialize(Serialization & ctx) const +{ + (void)ctx; +} + +std::unique_ptr UnionStep::deserialize(Deserialization & ctx) +{ + return std::make_unique(ctx.input_streams); +} + +void registerUnionStep(QueryPlanStepRegistry & registry) +{ + registry.registerStep("Union", &UnionStep::deserialize); +} + } diff --git a/src/Processors/QueryPlan/UnionStep.h b/src/Processors/QueryPlan/UnionStep.h index 4ab08785b01..4869af28c02 100644 --- a/src/Processors/QueryPlan/UnionStep.h +++ b/src/Processors/QueryPlan/UnionStep.h @@ -13,7 +13,7 @@ public: String getName() const override { return "Union"; } - QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings &) override; + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders pipelines, const BuildQueryPipelineSettings & settings) override; void describePipeline(FormatSettings & settings) const override; @@ -21,6 +21,9 @@ public: bool canUpdateInputStream() const override { return true; } + void serialize(Serialization & ctx) const override; + static std::unique_ptr deserialize(Deserialization & ctx); + private: void updateOutputStream() override; diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 6f8b3931803..dc4c8810634 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -52,10 +52,12 @@ RemoteQueryExecutor::RemoteQueryExecutor( const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_, + std::shared_ptr query_plan_, std::optional extension_, GetPriorityForLoadBalancing::Func priority_func_) : header(header_) , query(query_) + , query_plan(std::move(query_plan_)) , context(context_) , scalars(scalars_) , external_tables(external_tables_) @@ -75,7 +77,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( const Tables & external_tables_, QueryProcessingStage::Enum stage_, std::optional extension_) - : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_) + : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, nullptr, extension_) { create_connections = [this, pool, throttler, extension_](AsyncCallback) { @@ -132,7 +134,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( const Tables & external_tables_, QueryProcessingStage::Enum stage_, std::optional extension_) - : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_) + : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, nullptr, extension_) { create_connections = [this, &connection, throttler, extension_](AsyncCallback) { @@ -153,7 +155,7 @@ RemoteQueryExecutor::RemoteQueryExecutor( const Tables & external_tables_, QueryProcessingStage::Enum stage_, std::optional extension_) - : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_) + : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, nullptr, extension_) { create_connections = [this, connection_ptr, throttler, extension_](AsyncCallback) { @@ -173,8 +175,9 @@ RemoteQueryExecutor::RemoteQueryExecutor( const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_, + std::shared_ptr query_plan_, std::optional extension_) - : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_) + : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, std::move(query_plan_), extension_) { create_connections = [this, connections_, throttler, extension_](AsyncCallback) mutable { @@ -194,9 +197,10 @@ RemoteQueryExecutor::RemoteQueryExecutor( const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_, + std::shared_ptr query_plan_, std::optional extension_, GetPriorityForLoadBalancing::Func priority_func_) - : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_, priority_func_) + : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, std::move(query_plan_), extension_, priority_func_) { create_connections = [this, pool, throttler](AsyncCallback async_callback)->std::unique_ptr { @@ -397,6 +401,10 @@ void RemoteQueryExecutor::sendQueryUnlocked(ClientInfo::QueryKind query_kind, As if (settings.enable_scalar_subquery_optimization) sendScalars(); + + if (query_plan) + connections->sendQueryPlan(*query_plan); + sendExternalTables(); } diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 83f33607dbf..ddcc7774777 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -73,6 +73,7 @@ public: const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, + std::optional extension_ = std::nullopt); /// Takes already set connection. @@ -97,6 +98,7 @@ public: const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, + std::shared_ptr query_plan_ = nullptr, std::optional extension_ = std::nullopt); /// Takes a pool and gets one or several connections from it. @@ -109,6 +111,7 @@ public: const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, + std::shared_ptr query_plan_ = nullptr, std::optional extension_ = std::nullopt, GetPriorityForLoadBalancing::Func priority_func = {}); @@ -232,6 +235,7 @@ private: const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_, + std::shared_ptr query_plan_, std::optional extension_, GetPriorityForLoadBalancing::Func priority_func = {}); @@ -244,6 +248,7 @@ private: std::unique_ptr read_context; const String query; + std::shared_ptr query_plan; String query_id; ContextPtr context; diff --git a/src/QueryPipeline/SizeLimits.cpp b/src/QueryPipeline/SizeLimits.cpp index 4161f3f365f..3e86ae7e9f9 100644 --- a/src/QueryPipeline/SizeLimits.cpp +++ b/src/QueryPipeline/SizeLimits.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include namespace ProfileEvents @@ -14,6 +16,12 @@ namespace ProfileEvents namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_DATA; +} + bool SizeLimits::check(UInt64 rows, UInt64 bytes, const char * what, int too_many_rows_exception_code, int too_many_bytes_exception_code) const { if (overflow_mode == OverflowMode::THROW) @@ -64,4 +72,25 @@ bool SizeLimits::check(UInt64 rows, UInt64 bytes, const char * what, int excepti return check(rows, bytes, what, exception_code, exception_code); } +static void checkAllowedOwerflowMode(OverflowMode mode, int code) +{ + if (!(mode == OverflowMode::BREAK || mode == OverflowMode::THROW)) + throw Exception(code, "Unexpected overflow mode {}", mode); +} + +void SizeLimits::serialize(WriteBuffer & out) const +{ + checkAllowedOwerflowMode(overflow_mode, ErrorCodes::LOGICAL_ERROR); + writeVarUInt(max_rows, out); + writeVarUInt(max_bytes, out); + writeIntBinary(overflow_mode, out); +} +void SizeLimits::deserialize(ReadBuffer & in) +{ + checkAllowedOwerflowMode(overflow_mode, ErrorCodes::INCORRECT_DATA); + readVarUInt(max_rows, in); + readVarUInt(max_bytes, in); + readIntBinary(overflow_mode, in); +} + } diff --git a/src/QueryPipeline/SizeLimits.h b/src/QueryPipeline/SizeLimits.h index 1c84f81a127..04293ee8966 100644 --- a/src/QueryPipeline/SizeLimits.h +++ b/src/QueryPipeline/SizeLimits.h @@ -18,6 +18,9 @@ enum class OverflowMode : uint8_t ANY = 2, }; +class WriteBuffer; +class ReadBuffer; + struct SizeLimits { @@ -38,6 +41,9 @@ struct SizeLimits bool softCheck(UInt64 rows, UInt64 bytes) const; bool hasLimits() const { return max_rows || max_bytes; } + + void serialize(WriteBuffer & out) const; + void deserialize(ReadBuffer & in); }; } diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 9c8e0c6bf73..a9f862a1743 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -977,7 +977,7 @@ namespace query_end = insert_query->data; } String query(begin, query_end); - io = ::DB::executeQuery(query, query_context).second; + io = ::DB::executeQuery(query, nullptr, query_context).second; } void Call::processInput() diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 50963887752..0afc1860d99 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -49,6 +49,7 @@ #include #include #include +#include #if USE_SSL # include @@ -561,7 +562,7 @@ void TCPHandler::runImpl() }); /// Processing Query - std::tie(state.parsed_query, state.io) = executeQuery(state.query, query_context, QueryFlags{}, state.stage); + std::tie(state.parsed_query, state.io) = executeQuery(state.query, state.plan_and_sets, query_context, QueryFlags{}, state.stage); after_check_cancelled.restart(); after_send_progress.restart(); @@ -1747,13 +1748,15 @@ bool TCPHandler::receivePacket() receiveQuery(); return true; - case Protocol::Client::Data: case Protocol::Client::Scalar: - if (state.skipping_data) - return receiveUnexpectedData(false); - if (state.empty()) - receiveUnexpectedData(true); - return receiveData(packet_type == Protocol::Client::Scalar); + return receiveData(/*scalar=*/ true); + + case Protocol::Client::QueryPlan: + receiveQueryPlan(); + return true; + + case Protocol::Client::Data: + return receiveData(/*scalar=*/ false); case Protocol::Client::Ping: writeVarUInt(Protocol::Server::Pong, *out); @@ -2079,8 +2082,26 @@ void TCPHandler::receiveUnexpectedQuery() throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet Query received from client"); } +void TCPHandler::receiveQueryPlan() +{ + bool unexpected_packet = state.empty() || state.stage != QueryProcessingStage::QueryPlan || state.plan_and_sets || !query_context || state.read_all_data; + auto context = unexpected_packet ? Context::getGlobalContextInstance() : query_context; + + auto plan_and_sets = QueryPlan::deserialize(*in, context); + + if (unexpected_packet) + throw NetException(ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT, "Unexpected packet QueryPlan received from client"); + + state.plan_and_sets = std::make_shared(std::move(plan_and_sets)); +} + bool TCPHandler::receiveData(bool scalar) { + if (state.skipping_data) + return receiveUnexpectedData(false); + if (state.empty()) + return receiveUnexpectedData(true); + initBlockInput(); /// The name of the temporary table for writing data, default to empty string diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 3b6e0059a30..083d5954fe9 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -21,6 +21,7 @@ #include #include +#include "Client/IServerConnection.h" #include "Core/Types.h" #include "IServer.h" #include "Interpreters/AsynchronousInsertQueue.h" @@ -41,6 +42,7 @@ namespace DB class Session; struct Settings; +struct QueryPlanAndSets; class ColumnsDescription; struct ProfileInfo; class TCPServer; @@ -76,6 +78,7 @@ struct QueryState /// Query text. String query; + std::shared_ptr plan_and_sets; /// Parsed query ASTPtr parsed_query; /// Streams of blocks, that are processing the query. @@ -264,6 +267,7 @@ private: void receiveAddendum(); bool receivePacket(); void receiveQuery(); + void receiveQueryPlan(); void receiveIgnoredPartUUIDs(); String receiveReadTaskResponseAssumeLocked(); std::optional receivePartitionMergeTreeReadTaskResponseAssumeLocked(); diff --git a/src/Storages/IStorageCluster.cpp b/src/Storages/IStorageCluster.cpp index 63467603d16..d06e7df2fa2 100644 --- a/src/Storages/IStorageCluster.cpp +++ b/src/Storages/IStorageCluster.cpp @@ -190,6 +190,7 @@ void ReadFromCluster::initializePipeline(QueryPipelineBuilder & pipeline, const scalars, Tables(), processed_stage, + nullptr, extension); remote_query_executor->setLogger(log); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 0b80858800b..9d125beb0fc 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1132,6 +1132,7 @@ std::optional StorageDistributed::distributedWriteFromClusterStor Scalars{}, Tables{}, QueryProcessingStage::Complete, + nullptr, extension); QueryPipeline remote_pipeline(std::make_shared(remote_query_executor, false, settings.async_socket_for_remote, settings.async_query_sending_for_remote)); diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 1408e120bc5..1bf304503d4 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -194,7 +194,6 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables( { /// Execute remote query without restrictions (because it's not real user query, but part of implementation) RemoteQueryExecutor executor(shard_info.pool, query, sample_block, new_context); - executor.setPoolMode(PoolMode::GET_ONE); executor.setMainTable(remote_table_id);