From 65d23af611f28fbe0f2baf1dac6cd1aa317676b5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 24 Apr 2023 19:35:49 +0000 Subject: [PATCH 001/171] Refactor PreparedSets [1] --- src/Interpreters/PreparedSets.cpp | 101 +++++++++++-------- src/Interpreters/PreparedSets.h | 157 ++++++++++++++++++++++++++---- 2 files changed, 200 insertions(+), 58 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 7b0efddae87..11af41cae8f 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -5,6 +5,8 @@ #include #include #include +#include "Common/logger_useful.h" +#include "Processors/QueryPlan/CreatingSetsStep.h" namespace DB { @@ -66,30 +68,30 @@ String PreparedSetKey::toString() const return buf.str(); } -SubqueryForSet & PreparedSets::createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, - SizeLimits set_size_limit, bool transform_null_in) -{ - SubqueryForSet & subquery = subqueries[subquery_id]; +// SubqueryForSet & PreparedSets::createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, +// SizeLimits set_size_limit, bool transform_null_in) +// { +// SubqueryForSet & subquery = subqueries[subquery_id]; - /// If you already created a Set with the same subquery / table for another ast - /// In that case several PreparedSetKey would share same subquery and set - /// Not sure if it's really possible case (maybe for distributed query when set was filled by external table?) - if (subquery.set.isValid()) - sets[key] = subquery.set; - else - { - subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); - sets[key] = FutureSet(subquery.promise_to_fill_set.get_future()); - } +// /// If you already created a Set with the same subquery / table for another ast +// /// In that case several PreparedSetKey would share same subquery and set +// /// Not sure if it's really possible case (maybe for distributed query when set was filled by external table?) +// if (subquery.set.isValid()) +// sets[key] = subquery.set; +// else +// { +// subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); +// sets[key] = FutureSet(subquery.promise_to_fill_set.get_future()); +// } - if (!subquery.set_in_progress) - { - subquery.key = key.toString(); - subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); - } +// if (!subquery.set_in_progress) +// { +// subquery.key = key.toString(); +// subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); +// } - return subquery; -} +// return subquery; +// } /// If the subquery is not associated with any set, create default-constructed SubqueryForSet. /// It's aimed to fill external table passed to SubqueryForSet::createSource. @@ -154,26 +156,6 @@ QueryPlanPtr SubqueryForSet::detachSource() } -FutureSet::FutureSet(SetPtr set) -{ - std::promise promise; - promise.set_value(set); - *this = FutureSet(promise.get_future()); -} - - -bool FutureSet::isReady() const -{ - return future_set.valid() && - future_set.wait_for(std::chrono::seconds(0)) == std::future_status::ready; -} - -bool FutureSet::isCreated() const -{ - return isReady() && get() != nullptr && get()->isCreated(); -} - - std::variant, SharedSet> PreparedSetsCache::findOrPromiseToBuild(const String & key) { std::lock_guard lock(cache_mutex); @@ -194,4 +176,41 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo return promise_to_fill_set; } +std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) +{ + if (set) + return nullptr; + + auto set_cache = context->getPreparedSetsCache(); + if (set_cache) + { + auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); + if (from_cache.index() == 0) + { + subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); + } + else + { + LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); + set = std::get<1>(from_cache).get(); + return nullptr; + } + } + + subquery.set = set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + + auto plan = subquery.detachSource(); + + const Settings & settings = context->getSettingsRef(); + auto creating_set = std::make_unique( + plan->getCurrentDataStream(), + subquery_id, + std::move(subquery), + SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), + context); + creating_set->setStepDescription("Create set for subquery"); + plan->addStep(std::move(creating_set)); + return plan; +} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 4a7d1c3de46..a4898fe5ec6 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -10,6 +10,14 @@ #include #include #include +#include "Core/Block.h" +#include "Interpreters/Context.h" +#include "Interpreters/Set.h" +#include "Processors/Executors/CompletedPipelineExecutor.h" +#include "Processors/QueryPlan/BuildQueryPipelineSettings.h" +#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" +#include "Processors/Sinks/NullSink.h" +#include namespace DB { @@ -25,30 +33,83 @@ class InterpreterSelectWithUnionQuery; /// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there /// must be an explicit step to build them before they can be used. /// FutureSet objects can be stored in PreparedSets and are not intended to be used from multiple threads. -class FutureSet final +// class FutureSet final +// { +// public: +// FutureSet() = default; + +// /// Create FutureSet from an object that will be created in the future. +// explicit FutureSet(const std::shared_future & future_set_) : future_set(future_set_) {} + +// /// Create FutureSet from a ready set. +// explicit FutureSet(SetPtr readySet); + +// /// The set object will be ready in the future, as opposed to 'null' object when FutureSet is default constructed. +// bool isValid() const { return future_set.valid(); } + +// /// The the value of SetPtr is ready, but the set object might not have been filled yet. +// bool isReady() const; + +// /// The set object is ready and filled. +// bool isCreated() const; + +// SetPtr get() const { chassert(isReady()); return future_set.get(); } + +// private: +// std::shared_future future_set; +// }; + +class FutureSet { public: - FutureSet() = default; + virtual ~FutureSet() = default; - /// Create FutureSet from an object that will be created in the future. - explicit FutureSet(const std::shared_future & future_set_) : future_set(future_set_) {} + virtual bool isReady() const = 0; + virtual SetPtr get() const = 0; - /// Create FutureSet from a ready set. - explicit FutureSet(SetPtr readySet); + virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; + virtual std::unique_ptr build(const ContextPtr & context) = 0; +}; - /// The set object will be ready in the future, as opposed to 'null' object when FutureSet is default constructed. - bool isValid() const { return future_set.valid(); } +using FutureSetPtr = std::unique_ptr; - /// The the value of SetPtr is ready, but the set object might not have been filled yet. - bool isReady() const; +class FutureSetFromTuple final : public FutureSet +{ +public: + FutureSetFromTuple(Block block_, const SizeLimits & size_limits_, bool transform_null_in_); - /// The set object is ready and filled. - bool isCreated() const; + bool isReady() const override { return set != nullptr; } + SetPtr get() const override { return set; } - SetPtr get() const { chassert(isReady()); return future_set.get(); } + SetPtr buildOrderedSetInplace(const ContextPtr &) override + { + fill(true); + return set; + } + + std::unique_ptr build(const ContextPtr &) override + { + fill(false); + return nullptr; + } private: - std::shared_future future_set; + Block block; + SizeLimits size_limits; + bool transform_null_in; + + SetPtr set; + + void fill(bool create_ordered_set) + { + if (set) + return; + + set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); + set->insertFromBlock(block.getColumnsWithTypeAndName()); + set->finishInsert(); + } }; /// Information on how to build set for the [GLOBAL] IN section. @@ -66,11 +127,12 @@ public: /// Build this set from the result of the subquery. String key; - SetPtr set_in_progress; + SetPtr set; /// After set_in_progress is finished it will be put into promise_to_fill_set and thus all FutureSet's /// that are referencing this set will be filled. + std::promise promise_to_fill_set; - FutureSet set = FutureSet{promise_to_fill_set.get_future()}; + // FutureSet set = FutureSet{promise_to_fill_set.get_future()}; /// If set, put the result into the table. /// This is a temporary table for transferring to remote servers for distributed query processing. @@ -80,6 +142,67 @@ public: std::unique_ptr source; }; +class FutureSetFromSubquery : public FutureSet +{ +public: + FutureSetFromSubquery(SubqueryForSet subquery_, String subquery_id, SizeLimits set_size_limit_, bool transform_null_in_); + + bool isReady() const override { return set != nullptr; } + SetPtr get() const override { return set; } + + SetPtr buildOrderedSetInplace(const ContextPtr & context) override + { + auto plan = buildPlan(context, true); + + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + + return set; + } + + std::unique_ptr build(const ContextPtr & context) override + { + return buildPlan(context, false); + } + +private: + SetPtr set; + SubqueryForSet subquery; + String subquery_id; + SizeLimits size_limits; + bool transform_null_in; + + std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); +}; + +// class FutureSetFromFuture : public FutureSet +// { +// public: +// FutureSetFromFuture(std::shared_future future_set_); + +// bool isReady() const override { return future_set.wait_for(std::chrono::seconds(0)) == std::future_status::ready; } +// SetPtr get() const override { return future_set.get(); } + +// SetPtr buildOrderedSetInplace(const ContextPtr &) override +// { +// fill(true); +// return set; +// } + +// std::unique_ptr build(const ContextPtr &) override +// { +// fill(false); +// return nullptr; +// } + +// private: +// std::shared_future future_set; +// } + struct PreparedSetKey { /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired @@ -132,7 +255,7 @@ private: std::unordered_map sets; /// This is the information required for building sets - SubqueriesForSets subqueries; + // SubqueriesForSets subqueries; }; using PreparedSetsPtr = std::shared_ptr; From 80a2f30a0cc651db608735391d5173452f7b41ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 25 Apr 2023 18:14:08 +0000 Subject: [PATCH 002/171] Refactor PreparedSets [2] --- src/Columns/ColumnSet.h | 4 +- src/Interpreters/ActionsVisitor.cpp | 56 +++--- src/Interpreters/ActionsVisitor.h | 7 +- src/Interpreters/ExpressionAnalyzer.cpp | 188 ++++++++++----------- src/Interpreters/ExpressionAnalyzer.h | 4 +- src/Interpreters/GlobalSubqueriesVisitor.h | 24 +-- src/Interpreters/PreparedSets.cpp | 110 +++++++++--- src/Interpreters/PreparedSets.h | 75 +++++--- src/Planner/PlannerContext.h | 20 +-- 9 files changed, 283 insertions(+), 205 deletions(-) diff --git a/src/Columns/ColumnSet.h b/src/Columns/ColumnSet.h index 3f5cf4ad280..ccd9aa19896 100644 --- a/src/Columns/ColumnSet.h +++ b/src/Columns/ColumnSet.h @@ -21,7 +21,7 @@ class ColumnSet final : public COWHelper private: friend class COWHelper; - ColumnSet(size_t s_, FutureSet data_) : data(std::move(data_)) { s = s_; } + ColumnSet(size_t s_, FutureSetPtr data_) : data(std::move(data_)) { s = s_; } ColumnSet(const ColumnSet &) = default; public: @@ -35,7 +35,7 @@ public: Field operator[](size_t) const override { return {}; } private: - FutureSet data; + FutureSetPtr data; }; } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 3bb3ea67e29..00feecb44f5 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -422,9 +422,8 @@ Block createBlockForSet( } -SetPtr makeExplicitSet( - const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, - ContextPtr context, const SizeLimits & size_limits, PreparedSets & prepared_sets) +FutureSetPtr makeExplicitSet( + const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets) { const IAST & args = *node->arguments; @@ -448,7 +447,7 @@ SetPtr makeExplicitSet( element_type = low_cardinality_type->getDictionaryType(); auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types); - if (auto set = prepared_sets.get(set_key)) + if (auto set = prepared_sets.getFuture(set_key)) return set; /// Already prepared. Block block; @@ -458,14 +457,7 @@ SetPtr makeExplicitSet( else block = createBlockForSet(left_arg_type, right_arg, set_element_types, context); - SetPtr set - = std::make_shared(size_limits, create_ordered_set, context->getSettingsRef().transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(block.getColumnsWithTypeAndName()); - set->finishInsert(); - - prepared_sets.set(set_key, set); - return set; + return prepared_sets.addFromTuple(set_key, block); } class ScopeStack::Index @@ -952,7 +944,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & return; } - FutureSet prepared_set; + FutureSetPtr prepared_set; if (checkFunctionIsInOrGlobalInOperator(node)) { /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). @@ -961,7 +953,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (!data.no_makeset && !(data.is_create_parameterized_view && !analyzeReceiveQueryParams(ast).empty())) prepared_set = makeSet(node, data, data.no_subqueries); - if (prepared_set.isValid()) + if (prepared_set) { /// Transform tuple or subquery into a set. } @@ -1174,14 +1166,14 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & num_arguments += columns.size() - 1; arg += columns.size() - 1; } - else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set.isValid()) + else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set) { ColumnWithTypeAndName column; column.type = std::make_shared(); /// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name, /// so that sets with the same literal representation do not fuse together (they can have different types). - const bool is_constant_set = prepared_set.isCreated(); + const bool is_constant_set = typeid_cast(prepared_set.get()) == nullptr; if (is_constant_set) column.name = data.getUniqueName("__set"); else @@ -1373,7 +1365,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */, data.addColumn(std::move(column)); } -FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries) +FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries) { if (!data.prepared_sets) return {}; @@ -1394,11 +1386,8 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no return {}; auto set_key = PreparedSetKey::forSubquery(*right_in_operand); - { - auto set = data.prepared_sets->getFuture(set_key); - if (set.isValid()) - return set; - } + if (auto set = data.prepared_sets->getFuture(set_key)) + return set; /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). @@ -1409,20 +1398,16 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no if (table) { - StorageSet * storage_set = dynamic_cast(table.get()); - if (storage_set) - { - SetPtr set = storage_set->getSet(); - data.prepared_sets->set(set_key, set); - return FutureSet(set); - } + if (StorageSet * storage_set = dynamic_cast(table.get())) + return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. - String set_id = right_in_operand->getColumnName(); - bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; - SubqueryForSet & subquery_for_set = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); + // String set_id = right_in_operand->getColumnName(); + //bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; + SubqueryForSet subquery_for_set; // = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); + subquery_for_set.key = right_in_operand->getColumnName(); /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, @@ -1432,13 +1417,13 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no * In case that we have HAVING with IN subquery, we have to force creating set for it. * Also it doesn't make sense if it is GLOBAL IN or ordinary IN. */ - if (data.create_source_for_in && !subquery_for_set.hasSource()) + if (data.create_source_for_in) { auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); subquery_for_set.createSource(*interpreter); } - return subquery_for_set.set; + return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); } else { @@ -1446,8 +1431,7 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no const auto & index = data.actions_stack.getLastActionsIndex(); if (data.prepared_sets && index.contains(left_in_operand->getColumnName())) /// An explicit enumeration of values in parentheses. - return FutureSet( - makeExplicitSet(&node, last_actions, false, data.getContext(), data.set_size_limit, *data.prepared_sets)); + return makeExplicitSet(&node, last_actions, data.getContext(), *data.prepared_sets); else return {}; } diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index 260fd5ab2c0..71d57620196 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -25,9 +25,8 @@ class IFunctionOverloadResolver; using FunctionOverloadResolverPtr = std::shared_ptr; /// The case of an explicit enumeration of values. -SetPtr makeExplicitSet( - const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set, - ContextPtr context, const SizeLimits & limits, PreparedSets & prepared_sets); +FutureSetPtr makeExplicitSet( + const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets); /** For ActionsVisitor * A stack of ExpressionActions corresponding to nested lambda expressions. @@ -219,7 +218,7 @@ private: static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data); static void visit(ASTExpressionList & expression_list, const ASTPtr & ast, Data & data); - static FutureSet makeSet(const ASTFunction & node, Data & data, bool no_subqueries); + static FutureSetPtr makeSet(const ASTFunction & node, Data & data, bool no_subqueries); static ASTs doUntuple(const ASTFunction * function, ActionsMatcher::Data & data); static std::optional getNameAndTypeFromAST(const ASTPtr & ast, Data & data); }; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index cc54e7620f6..7dece81734c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -450,76 +450,76 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global, b } -void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options) -{ - if (!prepared_sets) - return; +// void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options) +// { +// if (!prepared_sets) +// return; - auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); +// auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); - if (prepared_sets->getFuture(set_key).isValid()) - return; /// Already prepared. +// if (prepared_sets->getFuture(set_key).isValid()) +// return; /// Already prepared. - if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name)) - { - prepared_sets->set(set_key, set_ptr_from_storage_set); - return; - } +// if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name)) +// { +// prepared_sets->set(set_key, set_ptr_from_storage_set); +// return; +// } - auto build_set = [&] () -> SetPtr - { - LOG_TRACE(getLogger(), "Building set, key: {}", set_key.toString()); +// auto build_set = [&] () -> SetPtr +// { +// LOG_TRACE(getLogger(), "Building set, key: {}", set_key.toString()); - auto interpreter_subquery = interpretSubquery(subquery_or_table_name, getContext(), {}, query_options); - auto io = interpreter_subquery->execute(); - PullingAsyncPipelineExecutor executor(io.pipeline); +// auto interpreter_subquery = interpretSubquery(subquery_or_table_name, getContext(), {}, query_options); +// auto io = interpreter_subquery->execute(); +// PullingAsyncPipelineExecutor executor(io.pipeline); - SetPtr set = std::make_shared(settings.size_limits_for_set_used_with_index, true, getContext()->getSettingsRef().transform_null_in); - set->setHeader(executor.getHeader().getColumnsWithTypeAndName()); +// SetPtr set = std::make_shared(settings.size_limits_for_set_used_with_index, true, getContext()->getSettingsRef().transform_null_in); +// set->setHeader(executor.getHeader().getColumnsWithTypeAndName()); - Block block; - while (executor.pull(block)) - { - if (block.rows() == 0) - continue; +// Block block; +// while (executor.pull(block)) +// { +// if (block.rows() == 0) +// continue; - /// If the limits have been exceeded, give up and let the default subquery processing actions take place. - if (!set->insertFromBlock(block.getColumnsWithTypeAndName())) - return nullptr; - } +// /// If the limits have been exceeded, give up and let the default subquery processing actions take place. +// if (!set->insertFromBlock(block.getColumnsWithTypeAndName())) +// return nullptr; +// } - set->finishInsert(); +// set->finishInsert(); - return set; - }; +// return set; +// }; - SetPtr set; +// SetPtr set; - auto set_cache = getContext()->getPreparedSetsCache(); - if (set_cache) - { - auto from_cache = set_cache->findOrPromiseToBuild(set_key.toString()); - if (from_cache.index() == 0) - { - set = build_set(); - std::get<0>(from_cache).set_value(set); - } - else - { - LOG_TRACE(getLogger(), "Waiting for set, key: {}", set_key.toString()); - set = std::get<1>(from_cache).get(); - } - } - else - { - set = build_set(); - } +// auto set_cache = getContext()->getPreparedSetsCache(); +// if (set_cache) +// { +// auto from_cache = set_cache->findOrPromiseToBuild(set_key.toString()); +// if (from_cache.index() == 0) +// { +// set = build_set(); +// std::get<0>(from_cache).set_value(set); +// } +// else +// { +// LOG_TRACE(getLogger(), "Waiting for set, key: {}", set_key.toString()); +// set = std::get<1>(from_cache).get(); +// } +// } +// else +// { +// set = build_set(); +// } - if (!set) - return; +// if (!set) +// return; - prepared_sets->set(set_key, std::move(set)); -} +// prepared_sets->set(set_key, std::move(set)); +// } SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name) { @@ -536,50 +536,50 @@ SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_o /// Performance optimization for IN() if storage supports it. -void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) -{ - if (!node || !storage() || !storage()->supportsIndexForIn()) - return; +// void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) +// { +// if (!node || !storage() || !storage()->supportsIndexForIn()) +// return; - for (auto & child : node->children) - { - /// Don't descend into subqueries. - if (child->as()) - continue; +// for (auto & child : node->children) +// { +// /// Don't descend into subqueries. +// if (child->as()) +// continue; - /// Don't descend into lambda functions - const auto * func = child->as(); - if (func && func->name == "lambda") - continue; +// /// Don't descend into lambda functions +// const auto * func = child->as(); +// if (func && func->name == "lambda") +// continue; - makeSetsForIndex(child); - } +// makeSetsForIndex(child); +// } - const auto * func = node->as(); - if (func && functionIsInOrGlobalInOperator(func->name)) - { - const IAST & args = *func->arguments; - const ASTPtr & left_in_operand = args.children.at(0); +// const auto * func = node->as(); +// if (func && functionIsInOrGlobalInOperator(func->name)) +// { +// const IAST & args = *func->arguments; +// const ASTPtr & left_in_operand = args.children.at(0); - if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot)) - { - const ASTPtr & arg = args.children.at(1); - if (arg->as() || arg->as()) - { - if (settings.use_index_for_in_with_subqueries) - tryMakeSetForIndexFromSubquery(arg, query_options); - } - else - { - auto temp_actions = std::make_shared(columns_after_join); - getRootActions(left_in_operand, true, temp_actions); +// if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot)) +// { +// const ASTPtr & arg = args.children.at(1); +// if (arg->as() || arg->as()) +// { +// if (settings.use_index_for_in_with_subqueries) +// tryMakeSetForIndexFromSubquery(arg, query_options); +// } +// else +// { +// auto temp_actions = std::make_shared(columns_after_join); +// getRootActions(left_in_operand, true, temp_actions); - if (prepared_sets && temp_actions->tryFindInOutputs(left_in_operand->getColumnName())) - makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, *prepared_sets); - } - } - } -} +// if (prepared_sets && temp_actions->tryFindInOutputs(left_in_operand->getColumnName())) +// makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, *prepared_sets); +// } +// } +// } +// } void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 1b6e8e24091..e4926b80625 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -144,7 +144,7 @@ public: /** Create Set from a subquery or a table expression in the query. The created set is suitable for using the index. * The set will not be created if its size hits the limit. */ - void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {}); + // void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {}); /** Checks if subquery is not a plain StorageSet. * Because while making set we will read data from StorageSet which is not allowed. @@ -364,7 +364,7 @@ public: ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const; /// Create Set-s that we make from IN section to use index on them. - void makeSetsForIndex(const ASTPtr & node); + // void makeSetsForIndex(const ASTPtr & node); private: StorageMetadataPtr metadata_snapshot; diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index b105cae31c6..9fb2c02bd58 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -167,19 +167,21 @@ public: { /// Do not materialize external tables if it's explain statement. } - else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) - { - auto external_table = external_storage_holder->getTable(); - auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); - auto io = interpreter->execute(); - io.pipeline.complete(std::move(table_out)); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); - } + // else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) + // { + // auto external_table = external_storage_holder->getTable(); + // auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); + // auto io = interpreter->execute(); + // io.pipeline.complete(std::move(table_out)); + // CompletedPipelineExecutor executor(io.pipeline); + // executor.execute(); + // } else { - auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); - subquery_for_set.createSource(*interpreter, external_storage); + // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); + // subquery_for_set.createSource(*interpreter, external_storage); + auto key = subquery_or_table_name->getColumnName(); + prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 11af41cae8f..300ef2aadb6 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -95,11 +95,51 @@ String PreparedSetKey::toString() const /// If the subquery is not associated with any set, create default-constructed SubqueryForSet. /// It's aimed to fill external table passed to SubqueryForSet::createSource. -SubqueryForSet & PreparedSets::getSubquery(const String & subquery_id) { return subqueries[subquery_id]; } +void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) +{ + auto it = subqueries.find(subquery_id); + if (it == subqueries.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); -void PreparedSets::set(const PreparedSetKey & key, SetPtr set_) { sets[key] = FutureSet(set_); } + it->second->addStorage(std::move(storage)); +} -FutureSet PreparedSets::getFuture(const PreparedSetKey & key) const +FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set_) +{ + auto from_storage = std::make_shared(std::move(set_)); + auto [it, inserted] = sets.emplace(key, std::move(from_storage)); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + + return it->second; +} + +FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block) +{ + auto from_tuple = std::make_shared(std::move(block)); + auto [it, inserted] = sets.emplace(key, std::move(from_tuple)); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + + return it->second; +} + +FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery) +{ + auto id = subquery.key; + auto from_subquery = std::make_shared(std::move(subquery)); + auto [it, inserted] = sets.emplace(key, from_subquery); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + + subqueries.emplace(id, std::move(from_subquery)); + return it->second; +} + +FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const { auto it = sets.find(key); if (it == sets.end()) @@ -107,24 +147,24 @@ FutureSet PreparedSets::getFuture(const PreparedSetKey & key) const return it->second; } -SetPtr PreparedSets::get(const PreparedSetKey & key) const -{ - auto it = sets.find(key); - if (it == sets.end() || !it->second.isReady()) - return nullptr; - return it->second.get(); -} +// SetPtr PreparedSets::get(const PreparedSetKey & key) const +// { +// auto it = sets.find(key); +// if (it == sets.end() || !it->second.isReady()) +// return nullptr; +// return it->second.get(); +// } -std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const -{ - std::vector res; - for (const auto & it : this->sets) - { - if (it.first.ast_hash == ast_hash) - res.push_back(it.second); - } - return res; -} +// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const +// { +// std::vector res; +// for (const auto & it : this->sets) +// { +// if (it.first.ast_hash == ast_hash) +// res.push_back(it.second); +// } +// return res; +// } PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() { @@ -197,14 +237,18 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c } } - subquery.set = set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + + subquery.set = set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); auto plan = subquery.detachSource(); + auto description = subquery.key; - const Settings & settings = context->getSettingsRef(); auto creating_set = std::make_unique( plan->getCurrentDataStream(), - subquery_id, + description, std::move(subquery), SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), context); @@ -213,4 +257,24 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c return plan; } + +static SizeLimits getSizeLimitsForUnorderedSet(const Settings & settings) +{ + return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); +} + +static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) +{ + if (settings.use_index_for_in_with_subqueries_max_values && + settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) + return getSizeLimitsForUnorderedSet(settings); + + return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); +} + +SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) +{ + return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); +} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index a4898fe5ec6..8ebabc32b0a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -69,43 +69,46 @@ public: virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; virtual std::unique_ptr build(const ContextPtr & context) = 0; + + static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); }; -using FutureSetPtr = std::unique_ptr; +using FutureSetPtr = std::shared_ptr; class FutureSetFromTuple final : public FutureSet { public: - FutureSetFromTuple(Block block_, const SizeLimits & size_limits_, bool transform_null_in_); + FutureSetFromTuple(Block block_); bool isReady() const override { return set != nullptr; } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr &) override + SetPtr buildOrderedSetInplace(const ContextPtr & context) override { - fill(true); + fill(context, true); return set; } - std::unique_ptr build(const ContextPtr &) override + std::unique_ptr build(const ContextPtr & context) override { - fill(false); + fill(context, false); return nullptr; } private: Block block; - SizeLimits size_limits; - bool transform_null_in; SetPtr set; - void fill(bool create_ordered_set) + void fill(const ContextPtr & context, bool create_ordered_set) { if (set) return; - set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + + set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); set->insertFromBlock(block.getColumnsWithTypeAndName()); set->finishInsert(); @@ -145,13 +148,16 @@ public: class FutureSetFromSubquery : public FutureSet { public: - FutureSetFromSubquery(SubqueryForSet subquery_, String subquery_id, SizeLimits set_size_limit_, bool transform_null_in_); + FutureSetFromSubquery(SubqueryForSet subquery_); bool isReady() const override { return set != nullptr; } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override { + if (!context->getSettingsRef().use_index_for_in_with_subqueries) + return nullptr; + auto plan = buildPlan(context, true); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); @@ -169,16 +175,34 @@ public: return buildPlan(context, false); } + void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } + private: SetPtr set; SubqueryForSet subquery; - String subquery_id; - SizeLimits size_limits; - bool transform_null_in; std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; +class FutureSetFromStorage : public FutureSet +{ +public: + FutureSetFromStorage(SetPtr set_); // : set(std::move(set_) {} + + bool isReady() const override { return set != nullptr; } + SetPtr get() const override { return set; } + + SetPtr buildOrderedSetInplace(const ContextPtr &) override + { + return set->hasExplicitSetElements() ? set : nullptr; + } + + std::unique_ptr build(const ContextPtr &) override { return nullptr; } + +private: + SetPtr set; +}; + // class FutureSetFromFuture : public FutureSet // { // public: @@ -230,15 +254,20 @@ struct PreparedSetKey class PreparedSets { public: - using SubqueriesForSets = std::unordered_map; + using SubqueriesForSets = std::unordered_map>; - SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, - SizeLimits set_size_limit, bool transform_null_in); - SubqueryForSet & getSubquery(const String & subquery_id); + // SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, + // SizeLimits set_size_limit, bool transform_null_in); - void set(const PreparedSetKey & key, SetPtr set_); - FutureSet getFuture(const PreparedSetKey & key) const; - SetPtr get(const PreparedSetKey & key) const; + FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); + FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block); + FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery); + + void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); + + FutureSetPtr getFuture(const PreparedSetKey & key) const; + //SubqueryForSet & getSubquery(const String & subquery_id); + // SetPtr get(const PreparedSetKey & key) const; /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. @@ -252,10 +281,10 @@ public: bool empty() const; private: - std::unordered_map sets; + std::unordered_map sets; /// This is the information required for building sets - // SubqueriesForSets subqueries; + SubqueriesForSets subqueries; }; using PreparedSetsPtr = std::shared_ptr; diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index ccc4ab43638..9ecfdb6117a 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -57,18 +57,18 @@ class PlannerSet { public: /// Construct planner set that is ready for execution - explicit PlannerSet(FutureSet set_) + explicit PlannerSet(FutureSetPtr set_) : set(std::move(set_)) {} /// Construct planner set with set and subquery node explicit PlannerSet(QueryTreeNodePtr subquery_node_) - : set(promise_to_build_set.get_future()) - , subquery_node(std::move(subquery_node_)) + //: set(promise_to_build_set.get_future()) + : subquery_node(std::move(subquery_node_)) {} /// Get a reference to a set that might be not built yet - const FutureSet & getSet() const + const FutureSetPtr & getSet() const { return set; } @@ -80,14 +80,14 @@ public: } /// This promise will be fulfilled when set is built and all FutureSet objects will become ready - std::promise extractPromiseToBuildSet() - { - return std::move(promise_to_build_set); - } + // std::promise extractPromiseToBuildSet() + // { + // return std::move(promise_to_build_set); + // } private: - std::promise promise_to_build_set; - FutureSet set; + //std::promise promise_to_build_set; + FutureSetPtr set; QueryTreeNodePtr subquery_node; }; From f598a39ea261161cefa363f21112ddb49c7300a5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 4 May 2023 17:54:08 +0000 Subject: [PATCH 003/171] Refactor PreparedSets [3] --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 18 ++- src/Analyzer/SetUtils.cpp | 24 ++-- src/Analyzer/SetUtils.h | 3 +- src/Columns/ColumnSet.h | 2 +- src/DataTypes/DataTypeSet.h | 2 +- src/Functions/in.cpp | 21 +++- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/ExpressionActions.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 14 ++- src/Interpreters/InterpreterSelectQuery.cpp | 11 +- src/Interpreters/PreparedSets.cpp | 22 ++-- src/Interpreters/PreparedSets.h | 42 ++++--- src/Planner/CollectSets.cpp | 61 ++++++++-- src/Planner/CollectSets.h | 4 +- src/Planner/Planner.cpp | 90 ++++++++------- src/Planner/PlannerActionsVisitor.cpp | 35 ++++-- src/Planner/PlannerContext.cpp | 72 ++++++------ src/Planner/PlannerContext.h | 107 +++++++++--------- src/Planner/PlannerJoinTree.cpp | 21 ++-- src/Planner/Utils.cpp | 3 +- src/Planner/Utils.h | 1 + src/Processors/QueryPlan/CreatingSetsStep.cpp | 72 +++++++++--- src/Processors/QueryPlan/CreatingSetsStep.h | 19 +++- .../QueryPlan/Optimizations/Optimizations.h | 1 + .../Optimizations/addPlansForSets.cpp | 35 ++++++ .../QueryPlan/Optimizations/optimizeTree.cpp | 1 + src/Processors/QueryPlan/QueryPlan.cpp | 5 + src/Processors/QueryPlan/QueryPlan.h | 5 +- .../Transforms/CreatingSetsTransform.cpp | 24 ++-- src/Storages/KVStorageUtils.cpp | 13 ++- src/Storages/MergeTree/KeyCondition.cpp | 20 +++- .../MergeTreeIndexConditionBloomFilter.cpp | 8 +- .../MergeTree/MergeTreeIndexFullText.cpp | 9 +- .../MergeTree/MergeTreeIndexInverted.cpp | 9 +- src/Storages/MergeTree/RPNBuilder.cpp | 42 ++++--- src/Storages/MergeTree/RPNBuilder.h | 6 +- .../System/StorageSystemZooKeeper.cpp | 11 +- src/Storages/VirtualColumnUtils.cpp | 38 +++---- 38 files changed, 585 insertions(+), 292 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 8afb9078fae..78e8b1760cb 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5117,14 +5117,26 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi const auto & second_argument_constant_literal = second_argument_constant_node->getValue(); const auto & second_argument_constant_type = second_argument_constant_node->getResultType(); - auto set = makeSetForConstantValue(first_argument_constant_type, + const auto & settings = scope.context->getSettingsRef(); + + auto result_block = makeSetForConstantValue(first_argument_constant_type, second_argument_constant_literal, second_argument_constant_type, - scope.context->getSettingsRef()); + settings.transform_null_in); + + 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, true /*fill_set_elements*/, 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)); /// Create constant set column for constant folding - auto column_set = ColumnSet::create(1, FutureSet(std::move(set))); + auto column_set = ColumnSet::create(1, std::move(future_set)); argument_columns[1].column = ColumnConst::create(std::move(column_set), 1); } diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 0fb075e925b..2fb05449714 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -118,7 +118,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & } -SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings) +Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in) { DataTypes set_element_types = {expression_type}; const auto * lhs_tuple_type = typeid_cast(expression_type.get()); @@ -135,8 +135,8 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field size_t lhs_type_depth = getCompoundTypeDepth(*expression_type); size_t rhs_type_depth = getCompoundTypeDepth(*value_type); - SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - bool tranform_null_in = settings.transform_null_in; + // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + // bool transform_null_in = settings.transform_null_in; Block result_block; @@ -145,7 +145,7 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. Array array{value}; - result_block = createBlockFromCollection(array, set_element_types, tranform_null_in); + result_block = createBlockFromCollection(array, set_element_types, transform_null_in); } else if (lhs_type_depth + 1 == rhs_type_depth) { @@ -154,9 +154,9 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field WhichDataType rhs_which_type(value_type); if (rhs_which_type.isArray()) - result_block = createBlockFromCollection(value.get(), set_element_types, tranform_null_in); + result_block = createBlockFromCollection(value.get(), set_element_types, transform_null_in); else if (rhs_which_type.isTuple()) - result_block = createBlockFromCollection(value.get(), set_element_types, tranform_null_in); + result_block = createBlockFromCollection(value.get(), set_element_types, transform_null_in); else throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Unsupported type at the right-side of IN. Expected Array or Tuple. Actual {}", @@ -170,13 +170,15 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field value_type->getName()); } - auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, tranform_null_in); + return result_block; - set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(result_block.getColumnsWithTypeAndName()); - set->finishInsert(); + // auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, tranform_null_in); - return set; + // set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); + // set->insertFromBlock(result_block.getColumnsWithTypeAndName()); + // set->finishInsert(); + + // return set; } } diff --git a/src/Analyzer/SetUtils.h b/src/Analyzer/SetUtils.h index 7afc8e5259c..fdeaa4a3c48 100644 --- a/src/Analyzer/SetUtils.h +++ b/src/Analyzer/SetUtils.h @@ -21,10 +21,9 @@ using SetPtr = std::shared_ptr; * @param expression_type - type of first argument of function IN. * @param value - constant value of second argument of function IN. * @param value_type - type of second argument of function IN. - * @param settings - query settings. * * @return SetPtr for constant value. */ -SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings); +Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in); } diff --git a/src/Columns/ColumnSet.h b/src/Columns/ColumnSet.h index ccd9aa19896..935a72e551a 100644 --- a/src/Columns/ColumnSet.h +++ b/src/Columns/ColumnSet.h @@ -29,7 +29,7 @@ public: TypeIndex getDataType() const override { return TypeIndex::Set; } MutableColumnPtr cloneDummy(size_t s_) const override { return ColumnSet::create(s_, data); } - ConstSetPtr getData() const { if (!data.isReady()) return nullptr; return data.get(); } + FutureSetPtr getData() const { return data; } // Used only for debugging, making it DUMPABLE Field operator[](size_t) const override { return {}; } diff --git a/src/DataTypes/DataTypeSet.h b/src/DataTypes/DataTypeSet.h index 7ddfeb9fe30..7ef0d931279 100644 --- a/src/DataTypes/DataTypeSet.h +++ b/src/DataTypes/DataTypeSet.h @@ -20,7 +20,7 @@ public: bool isParametric() const override { return true; } // Used for expressions analysis. - MutableColumnPtr createColumn() const override { return ColumnSet::create(0, FutureSet{}); } + MutableColumnPtr createColumn() const override { return ColumnSet::create(0, nullptr); } // Used only for debugging, making it DUMPABLE Field getDefault() const override { return Tuple(); } diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index 7a41ae2e3ea..eb623951bf3 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -55,9 +55,13 @@ public: /// It is needed to perform type analysis without creation of set. static constexpr auto name = FunctionInName::name; - static FunctionPtr create(ContextPtr) + FunctionIn(SizeLimits size_limits_, bool transform_null_in_) + : size_limits(std::move(size_limits_)), transform_null_in(transform_null_in_) {} + + static FunctionPtr create(ContextPtr context) { - return std::make_shared(); + const auto & settings = context->getSettingsRef(); + return std::make_shared(FutureSet::getSizeLimitsForSet(settings, false), settings.transform_null_in); } String getName() const override @@ -122,10 +126,15 @@ public: tuple = typeid_cast(materialized_tuple.get()); } - auto set = column_set->getData(); - if (!set) + auto future_set = column_set->getData(); + if (!future_set || !future_set->isFilled()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName()); + if (auto * for_tuple = typeid_cast(future_set.get())) + if (!for_tuple->isReady()) + for_tuple->buildForTuple(size_limits, transform_null_in); + + auto set = future_set->get(); auto set_types = set->getDataTypes(); if (tuple && set_types.size() != 1 && set_types.size() == tuple->tupleSize()) @@ -173,6 +182,10 @@ public: return res; } + +private: + SizeLimits size_limits; + bool transform_null_in; }; template diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 00feecb44f5..18e44ab9f2f 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -446,7 +446,7 @@ FutureSetPtr makeExplicitSet( if (const auto * low_cardinality_type = typeid_cast(element_type.get())) element_type = low_cardinality_type->getDictionaryType(); - auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types); + auto set_key = PreparedSetKey::forLiteral(right_arg->getTreeHash(), set_element_types); if (auto set = prepared_sets.getFuture(set_key)) return set; /// Already prepared. @@ -1384,7 +1384,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool { if (no_subqueries) return {}; - auto set_key = PreparedSetKey::forSubquery(*right_in_operand); + auto set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); if (auto set = data.prepared_sets->getFuture(set_key)) return set; diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 36725f36804..989fa12cba0 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -941,7 +941,7 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con if (const auto * column_set = checkAndGetColumn(action.node->column.get())) { auto set = column_set->getData(); - if (set && set->isCreated() && set->getTotalRowCount() == 0) + if (set && set->isReady() && set->get()->getTotalRowCount() == 0) return true; } } diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 9fb2c02bd58..3a846bb4bc3 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -181,7 +181,19 @@ public: // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); - prepared_sets->addStorageToSubquery(key, std::move(external_storage)); + auto set_key = PreparedSetKey::forSubquery(subquery_or_table_name->getTreeHash()); + + if (!prepared_sets->getFuture(set_key)) + { + SubqueryForSet subquery_for_set; + subquery_for_set.key = std::move(key); + subquery_for_set.table = std::move(external_storage); + subquery_for_set.createSource(*interpreter); + + prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); + } + else + prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index afd34cb044e..2487381e2fb 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -905,8 +905,8 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (storage && !options.only_analyze) { - query_analyzer->makeSetsForIndex(select_query.where()); - query_analyzer->makeSetsForIndex(select_query.prewhere()); + // query_analyzer->makeSetsForIndex(select_query.where()); + // query_analyzer->makeSetsForIndex(select_query.prewhere()); query_info.prepared_sets = query_analyzer->getPreparedSets(); from_stage = storage->getQueryProcessingStage(context, options.to_stage, storage_snapshot, query_info); @@ -3088,7 +3088,12 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - addCreatingSetsStep(query_plan, prepared_sets, context); + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + prepared_sets->detachSubqueries(context), + context); + + query_plan.addStep(std::move(step)); } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 300ef2aadb6..733eb1c24bb 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -11,7 +11,7 @@ namespace DB { -PreparedSetKey PreparedSetKey::forLiteral(const IAST & ast, DataTypes types_) +PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) { /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, /// just converts LowCardinality to ordinary types. @@ -19,15 +19,15 @@ PreparedSetKey PreparedSetKey::forLiteral(const IAST & ast, DataTypes types_) type = recursiveRemoveLowCardinality(type); PreparedSetKey key; - key.ast_hash = ast.getTreeHash(); + key.ast_hash = hash; key.types = std::move(types_); return key; } -PreparedSetKey PreparedSetKey::forSubquery(const IAST & ast) +PreparedSetKey PreparedSetKey::forSubquery(Hash hash) { PreparedSetKey key; - key.ast_hash = ast.getTreeHash(); + key.ast_hash = hash; return key; } @@ -155,9 +155,9 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return it->second.get(); // } -// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const +// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const // { -// std::vector res; +// std::vector res; // for (const auto & it : this->sets) // { // if (it.first.ast_hash == ast_hash) @@ -166,7 +166,7 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return res; // } -PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() +PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries(const ContextPtr &) { auto res = std::move(subqueries); subqueries = SubqueriesForSets(); @@ -221,6 +221,8 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; + std::cerr << StackTrace().toString() << std::endl; + auto set_cache = context->getPreparedSetsCache(); if (set_cache) { @@ -277,4 +279,10 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } +FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} + +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} + +FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 8ebabc32b0a..8661d81a96a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -65,6 +65,7 @@ public: virtual ~FutureSet() = default; virtual bool isReady() const = 0; + virtual bool isFilled() const = 0; virtual SetPtr get() const = 0; virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; @@ -81,34 +82,41 @@ public: FutureSetFromTuple(Block block_); bool isReady() const override { return set != nullptr; } + bool isFilled() const override { return true; } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override { - fill(context, true); + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, true); + fill(size_limits, settings.transform_null_in, true); return set; } std::unique_ptr build(const ContextPtr & context) override { - fill(context, false); + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, false); + fill(size_limits, settings.transform_null_in, false); return nullptr; } + void buildForTuple(SizeLimits size_limits, bool transform_null_in) + { + fill(size_limits, transform_null_in, false); + } + private: Block block; SetPtr set; - void fill(const ContextPtr & context, bool create_ordered_set) + void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) { if (set) return; - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); - - set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + set = std::make_shared(size_limits, create_ordered_set, transform_null_in); set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); set->insertFromBlock(block.getColumnsWithTypeAndName()); set->finishInsert(); @@ -151,6 +159,7 @@ public: FutureSetFromSubquery(SubqueryForSet subquery_); bool isReady() const override { return set != nullptr; } + bool isFilled() const override { return isReady(); } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override @@ -190,6 +199,7 @@ public: FutureSetFromStorage(SetPtr set_); // : set(std::move(set_) {} bool isReady() const override { return set != nullptr; } + bool isFilled() const override { return isReady(); } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr &) override @@ -229,23 +239,25 @@ private: struct PreparedSetKey { + using Hash = std::pair; + /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired /// data types of set elements (two different Sets can be required for two tuples with the same contents /// if left hand sides of the IN operators have different types). - static PreparedSetKey forLiteral(const IAST & ast, DataTypes types_); + static PreparedSetKey forLiteral(Hash hash, DataTypes types_); /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting /// set is fully determined by the subquery. - static PreparedSetKey forSubquery(const IAST & ast); + static PreparedSetKey forSubquery(Hash hash); - IAST::Hash ast_hash; + Hash ast_hash; DataTypes types; /// Empty for subqueries. bool operator==(const PreparedSetKey & other) const; String toString() const; - struct Hash + struct Hashing { UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } }; @@ -272,16 +284,18 @@ public: /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. /// SetPtr would still be available for consumers of PreparedSets. - SubqueriesForSets detachSubqueries(); + SubqueriesForSets detachSubqueries(const ContextPtr &); /// Returns all sets that match the given ast hash not checking types /// Used in KeyCondition and MergeTreeIndexConditionBloomFilter to make non exact match for types in PreparedSetKey - std::vector getByTreeHash(IAST::Hash ast_hash) const; + //std::vector getByTreeHash(IAST::Hash ast_hash) const; + + const std::unordered_map & getSets() const { return sets; } bool empty() const; private: - std::unordered_map sets; + std::unordered_map sets; /// This is the information required for building sets SubqueriesForSets subqueries; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 02069aad292..6c970e0e91b 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -11,6 +11,8 @@ #include #include #include +#include +#include namespace DB { @@ -26,8 +28,9 @@ namespace class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor { public: - explicit CollectSetsVisitor(PlannerContext & planner_context_) + explicit CollectSetsVisitor(PlannerContext & planner_context_, const SelectQueryOptions & select_query_options_) : planner_context(planner_context_) + , select_query_options(select_query_options_) {} void visitImpl(const QueryTreeNodePtr & node) @@ -42,10 +45,12 @@ public: const auto & settings = planner_context.getQueryContext()->getSettingsRef(); - String set_key = planner_context.createSetKey(in_second_argument); + // String set_key = planner_context.createSetKey(in_second_argument); - if (planner_context.hasSet(set_key)) - return; + // if (planner_context.hasSet(set_key)) + // return; + + auto & sets = planner_context.getPreparedSets(); /// Tables and table functions are replaced with subquery at Analysis stage, except special Set table. auto * second_argument_table = in_second_argument->as(); @@ -54,7 +59,9 @@ public: if (storage_set) { /// Handle storage_set as ready set. - planner_context.registerSet(set_key, PlannerSet(FutureSet(storage_set->getSet()))); + auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + sets.addFromStorage(set_key, storage_set->getSet()); + //planner_context.registerSet(set_key, PlannerSet(FutureSet(storage_set->getSet()))); } else if (const auto * constant_node = in_second_argument->as()) { @@ -62,14 +69,47 @@ public: in_first_argument->getResultType(), constant_node->getValue(), constant_node->getResultType(), - settings); + settings.transform_null_in); - planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); + DataTypes set_element_types = {in_first_argument->getResultType()}; + const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + + auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + + sets.addFromTuple(set_key, std::move(set)); + + //planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); } else if (in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION) { - planner_context.registerSet(set_key, PlannerSet(in_second_argument)); + auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + + auto subquery_options = select_query_options.subquery(); + Planner subquery_planner( + in_second_argument, + subquery_options, + planner_context.getGlobalPlannerContext()); + subquery_planner.buildQueryPlanIfNeeded(); + + // const auto & settings = planner_context.getQueryContext()->getSettingsRef(); + // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; + // bool tranform_null_in = settings.transform_null_in; + // auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); + + SubqueryForSet subquery_for_set; + subquery_for_set.key = planner_context.createSetKey(in_second_argument); + subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + + sets.addFromSubquery(set_key, std::move(subquery_for_set)); + + //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } else { @@ -87,13 +127,14 @@ public: private: PlannerContext & planner_context; + const SelectQueryOptions & select_query_options; }; } -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options) { - CollectSetsVisitor visitor(planner_context); + CollectSetsVisitor visitor(planner_context, select_query_options); visitor.visit(node); } diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index 94f792e877b..57e662a392e 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -7,9 +7,11 @@ namespace DB { +struct SelectQueryOptions; + /** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations. * Collected sets are registered in planner context. */ -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d036c895fbb..38d0aa29d24 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -878,50 +878,50 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana query_plan.addStep(std::move(offsets_step)); } -void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, - const SelectQueryOptions & select_query_options, - const PlannerContextPtr & planner_context, - const std::vector & result_actions_to_execute) -{ - PreparedSets::SubqueriesForSets subqueries_for_sets; +// void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, +// const SelectQueryOptions & select_query_options, +// const PlannerContextPtr & planner_context, +// const std::vector & result_actions_to_execute) +// { +// PreparedSets::SubqueriesForSets subqueries_for_sets; - for (const auto & actions_to_execute : result_actions_to_execute) - { - for (const auto & node : actions_to_execute->getNodes()) - { - const auto & set_key = node.result_name; - auto * planner_set = planner_context->getSetOrNull(set_key); - if (!planner_set) - continue; +// for (const auto & actions_to_execute : result_actions_to_execute) +// { +// for (const auto & node : actions_to_execute->getNodes()) +// { +// const auto & set_key = node.result_name; +// auto * planner_set = planner_context->getSetOrNull(set_key); +// if (!planner_set) +// continue; - if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) - continue; +// if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) +// continue; - auto subquery_options = select_query_options.subquery(); - Planner subquery_planner( - planner_set->getSubqueryNode(), - subquery_options, - planner_context->getGlobalPlannerContext()); - subquery_planner.buildQueryPlanIfNeeded(); +// auto subquery_options = select_query_options.subquery(); +// Planner subquery_planner( +// planner_set->getSubqueryNode(), +// subquery_options, +// planner_context->getGlobalPlannerContext()); +// subquery_planner.buildQueryPlanIfNeeded(); - const auto & settings = planner_context->getQueryContext()->getSettingsRef(); - SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - bool tranform_null_in = settings.transform_null_in; - auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); +// const auto & settings = planner_context->getQueryContext()->getSettingsRef(); +// SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; +// bool tranform_null_in = settings.transform_null_in; +// auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); - SubqueryForSet subquery_for_set; - subquery_for_set.key = set_key; - subquery_for_set.set_in_progress = set; - subquery_for_set.set = planner_set->getSet(); - subquery_for_set.promise_to_fill_set = planner_set->extractPromiseToBuildSet(); - subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); +// SubqueryForSet subquery_for_set; +// subquery_for_set.key = set_key; +// subquery_for_set.set_in_progress = set; +// subquery_for_set.set = planner_set->getSet(); +// subquery_for_set.promise_to_fill_set = planner_set->extractPromiseToBuildSet(); +// subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); - subqueries_for_sets.emplace(set_key, std::move(subquery_for_set)); - } - } +// subqueries_for_sets.emplace(set_key, std::move(subquery_for_set)); +// } +// } - addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); -} +// addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); +// } /// Support for `additional_result_filter` setting void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, @@ -951,7 +951,7 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, auto storage = std::make_shared(StorageID{"dummy", "dummy"}, fake_column_descriptions); auto fake_table_expression = std::make_shared(std::move(storage), query_context); - auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, std::move(fake_name_set)); + auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, select_query_options, std::move(fake_name_set)); if (!filter_info.actions || !query_plan.isInitialized()) return; @@ -1179,7 +1179,7 @@ void Planner::buildPlanForQueryNode() } checkStoragesSupportTransactions(planner_context); - collectSets(query_tree, *planner_context); + collectSets(query_tree, *planner_context, select_query_options); collectTableExpressionData(query_tree, planner_context); const auto & settings = query_context->getSettingsRef(); @@ -1467,7 +1467,17 @@ void Planner::buildPlanForQueryNode() } if (!select_query_options.only_analyze) - addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), + planner_context->getQueryContext()); + + query_plan.addStep(std::move(step)); + + //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); + //addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + } } SelectQueryInfo Planner::buildSelectQueryInfo() const diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index c64d82299ca..453b02a2f8f 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -623,33 +624,51 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::makeSetForInFunction(const QueryTreeNodePtr & node) { const auto & function_node = node->as(); + auto in_first_argument = function_node.getArguments().getNodes().at(0); auto in_second_argument = function_node.getArguments().getNodes().at(1); - auto set_key = planner_context->createSetKey(in_second_argument); - const auto & planner_set = planner_context->getSetOrThrow(set_key); + //auto set_key = planner_context->createSetKey(in_second_argument); + + DataTypes set_element_types = {in_first_argument->getResultType()}; + const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + + auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + + + auto set = planner_context->getPreparedSets().getFuture(set_key); + if (!set) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "No set is registered for key {}", + set_key.toString()); ColumnWithTypeAndName column; - column.name = set_key; + column.name = planner_context->createSetKey(in_second_argument); column.type = std::make_shared(); - bool set_is_created = planner_set.getSet().isCreated(); - auto column_set = ColumnSet::create(1, planner_set.getSet()); + bool set_is_created = set->isFilled(); + auto column_set = ColumnSet::create(1, std::move(set)); if (set_is_created) column.column = ColumnConst::create(std::move(column_set), 1); else column.column = std::move(column_set); - actions_stack[0].addConstantIfNecessary(set_key, column); + actions_stack[0].addConstantIfNecessary(column.name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(set_key, column); + actions_stack_node.addInputConstantColumnIfNecessary(column.name, column); } - return {set_key, 0}; + return {column.name, 0}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node) diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index 346cc6d2080..e8c7bb7ef48 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -126,49 +126,49 @@ PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); } -void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) -{ - if (!planner_set.getSet().isValid()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Set must be initialized"); +// void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) +// { +// if (!planner_set.getSet().isValid()) +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Set must be initialized"); - const auto & subquery_node = planner_set.getSubqueryNode(); - if (subquery_node) - { - auto node_type = subquery_node->getNodeType(); +// const auto & subquery_node = planner_set.getSubqueryNode(); +// if (subquery_node) +// { +// auto node_type = subquery_node->getNodeType(); - if (node_type != QueryTreeNodeType::QUERY && - node_type != QueryTreeNodeType::UNION) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Invalid node for set table expression. Expected query or union. Actual {}", - subquery_node->formatASTForErrorMessage()); - } +// if (node_type != QueryTreeNodeType::QUERY && +// node_type != QueryTreeNodeType::UNION) +// throw Exception(ErrorCodes::LOGICAL_ERROR, +// "Invalid node for set table expression. Expected query or union. Actual {}", +// subquery_node->formatASTForErrorMessage()); +// } - set_key_to_set.emplace(key, std::move(planner_set)); -} +// set_key_to_set.emplace(key, std::move(planner_set)); +// } -bool PlannerContext::hasSet(const SetKey & key) const -{ - return set_key_to_set.contains(key); -} +// bool PlannerContext::hasSet(const SetKey & key) const +// { +// return set_key_to_set.contains(key); +// } -const PlannerSet & PlannerContext::getSetOrThrow(const SetKey & key) const -{ - auto it = set_key_to_set.find(key); - if (it == set_key_to_set.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "No set is registered for key {}", - key); +// const PlannerSet & PlannerContext::getSetOrThrow(const SetKey & key) const +// { +// auto it = set_key_to_set.find(key); +// if (it == set_key_to_set.end()) +// throw Exception(ErrorCodes::LOGICAL_ERROR, +// "No set is registered for key {}", +// key); - return it->second; -} +// return it->second; +// } -PlannerSet * PlannerContext::getSetOrNull(const SetKey & key) -{ - auto it = set_key_to_set.find(key); - if (it == set_key_to_set.end()) - return nullptr; +// PlannerSet * PlannerContext::getSetOrNull(const SetKey & key) +// { +// auto it = set_key_to_set.find(key); +// if (it == set_key_to_set.end()) +// return nullptr; - return &it->second; -} +// return &it->second; +// } } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 9ecfdb6117a..df3fad01824 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -55,43 +55,47 @@ using GlobalPlannerContextPtr = std::shared_ptr; */ class PlannerSet { -public: - /// Construct planner set that is ready for execution - explicit PlannerSet(FutureSetPtr set_) - : set(std::move(set_)) - {} - /// Construct planner set with set and subquery node - explicit PlannerSet(QueryTreeNodePtr subquery_node_) - //: set(promise_to_build_set.get_future()) - : subquery_node(std::move(subquery_node_)) - {} - - /// Get a reference to a set that might be not built yet - const FutureSetPtr & getSet() const - { - return set; - } - - /// Get subquery node - const QueryTreeNodePtr & getSubqueryNode() const - { - return subquery_node; - } - - /// This promise will be fulfilled when set is built and all FutureSet objects will become ready - // std::promise extractPromiseToBuildSet() - // { - // return std::move(promise_to_build_set); - // } - -private: - //std::promise promise_to_build_set; - FutureSetPtr set; - - QueryTreeNodePtr subquery_node; }; +// { +// public: +// /// Construct planner set that is ready for execution +// explicit PlannerSet(FutureSetPtr set_) +// : set(std::move(set_)) +// {} + +// /// Construct planner set with set and subquery node +// explicit PlannerSet(QueryTreeNodePtr subquery_node_) +// //: set(promise_to_build_set.get_future()) +// : subquery_node(std::move(subquery_node_)) +// {} + +// /// Get a reference to a set that might be not built yet +// const FutureSetPtr & getSet() const +// { +// return set; +// } + +// /// Get subquery node +// const QueryTreeNodePtr & getSubqueryNode() const +// { +// return subquery_node; +// } + +// /// This promise will be fulfilled when set is built and all FutureSet objects will become ready +// // std::promise extractPromiseToBuildSet() +// // { +// // return std::move(promise_to_build_set); +// // } + +// private: +// //std::promise promise_to_build_set; +// FutureSetPtr set; + +// QueryTreeNodePtr subquery_node; +// }; + class PlannerContext { public: @@ -179,28 +183,30 @@ public: using SetKey = std::string; - using SetKeyToSet = std::unordered_map; + // using SetKeyToSet = std::unordered_map; - /// Create set key for set source node + // /// Create set key for set source node static SetKey createSetKey(const QueryTreeNodePtr & set_source_node); - /// Register set for set key - void registerSet(const SetKey & key, PlannerSet planner_set); + // /// Register set for set key + // void registerSet(const SetKey & key, PlannerSet planner_set); - /// Returns true if set is registered for key, false otherwise - bool hasSet(const SetKey & key) const; + // /// Returns true if set is registered for key, false otherwise + // bool hasSet(const SetKey & key) const; - /// Get set for key, if no set is registered logical exception is thrown - const PlannerSet & getSetOrThrow(const SetKey & key) const; + // /// Get set for key, if no set is registered logical exception is thrown + // const PlannerSet & getSetOrThrow(const SetKey & key) const; - /// Get set for key, if no set is registered null is returned - PlannerSet * getSetOrNull(const SetKey & key); + // /// Get set for key, if no set is registered null is returned + // PlannerSet * getSetOrNull(const SetKey & key); - /// Get registered sets - const SetKeyToSet & getRegisteredSets() const - { - return set_key_to_set; - } + // /// Get registered sets + // const SetKeyToSet & getRegisteredSets() const + // { + // return set_key_to_set; + // } + + PreparedSets & getPreparedSets() { return prepared_sets; } private: /// Query context @@ -216,8 +222,7 @@ private: std::unordered_map table_expression_node_to_data; /// Set key to set - SetKeyToSet set_key_to_set; - + PreparedSets prepared_sets; }; using PlannerContextPtr = std::shared_ptr; diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 93a9945b1ca..77f53a6b7ac 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -388,7 +388,8 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) + PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options) { auto storage_id = storage->getStorageID(); const auto & query_context = planner_context->getQueryContext(); @@ -397,12 +398,13 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, if (!row_policy_filter) return {}; - return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context); + return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context, select_query_options); } FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) + PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -428,14 +430,15 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, *storage, query_context); - return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context); + return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context, select_query_options); } /// Apply filters from additional_table_filters setting FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, const String & table_expression_alias, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context) + PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -469,7 +472,7 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, return {}; table_expression_query_info.additional_filter_ast = additional_filter_ast; - return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context); + return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context, select_query_options); } JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, @@ -679,14 +682,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } }; - auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context); + auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, select_query_options); add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) { if (settings.parallel_replicas_count > 1) { - auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context); + auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context, select_query_options); add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); } else @@ -701,7 +704,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } const auto & table_expression_alias = table_expression->getAlias(); - auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context); + auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context, select_query_options); add_filter(additional_filters_info, "additional filter"); from_stage = storage->getQueryProcessingStage(query_context, select_query_options.to_stage, storage_snapshot, table_expression_query_info); diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 5c61b2fc2c7..2b4febf58ea 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -426,6 +426,7 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter) { const auto & query_context = planner_context->getQueryContext(); @@ -443,7 +444,7 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, } collectSourceColumns(filter_query_tree, planner_context); - collectSets(filter_query_tree, *planner_context); + collectSets(filter_query_tree, *planner_context, select_query_options); auto filter_actions_dag = std::make_shared(); diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index d9412800e61..8071e201f88 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -82,6 +82,7 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, + const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter = {}); ASTPtr parseAdditionalResultFilter(const Settings & settings); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 9eec3e90494..d77c6627994 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -38,16 +38,16 @@ CreatingSetStep::CreatingSetStep( SizeLimits network_transfer_limits_, ContextPtr context_) : ITransformingStep(input_stream_, Block{}, getTraits()) - , WithContext(context_) , description(std::move(description_)) , subquery_for_set(std::move(subquery_for_set_)) , network_transfer_limits(std::move(network_transfer_limits_)) + , context(std::move(context_)) { } void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, getContext()); + pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context); } void CreatingSetStep::updateOutputStream() @@ -60,7 +60,7 @@ void CreatingSetStep::describeActions(FormatSettings & settings) const String prefix(settings.offset, ' '); settings.out << prefix; - if (subquery_for_set.set_in_progress) + if (subquery_for_set.set) settings.out << "Set: "; settings.out << description << '\n'; @@ -68,7 +68,7 @@ void CreatingSetStep::describeActions(FormatSettings & settings) const void CreatingSetStep::describeActions(JSONBuilder::JSONMap & map) const { - if (subquery_for_set.set_in_progress) + if (subquery_for_set.set) map.add("Set", description); } @@ -130,22 +130,14 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets plans.emplace_back(std::make_unique(std::move(query_plan))); query_plan = QueryPlan(); - for (auto & [description, subquery_for_set] : subqueries_for_sets) + for (auto & [description, future_set] : subqueries_for_sets) { - if (!subquery_for_set.hasSource()) + if (future_set->isReady()) continue; - auto plan = subquery_for_set.detachSource(); - - const Settings & settings = context->getSettingsRef(); - auto creating_set = std::make_unique( - plan->getCurrentDataStream(), - description, - std::move(subquery_for_set), - SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), - context); - creating_set->setStepDescription("Create set for subquery"); - plan->addStep(std::move(creating_set)); + auto plan = future_set->build(context); + if (!plan) + continue; input_streams.emplace_back(plan->getCurrentDataStream()); plans.emplace_back(std::move(plan)); @@ -162,12 +154,56 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets query_plan.unitePlans(std::move(creating_sets), std::move(plans)); } +//void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context) + +std::vector> DelayedCreatingSetsStep::makePlansForSets(DelayedCreatingSetsStep && step) +{ + // DataStreams input_streams; + // input_streams.emplace_back(query_plan.getCurrentDataStream()); + + std::vector> plans; + // plans.emplace_back(std::make_unique(std::move(query_plan))); + // query_plan = QueryPlan(); + + for (auto & [description, future_set] : step.subqueries_for_sets) + { + if (future_set->isReady()) + continue; + + auto plan = future_set->build(step.context); + if (!plan) + continue; + + plan->optimize(QueryPlanOptimizationSettings::fromContext(step.context)); + + //input_streams.emplace_back(plan->getCurrentDataStream()); + plans.emplace_back(std::move(plan)); + } + + return plans; +} + void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context) { if (!prepared_sets || prepared_sets->empty()) return; - addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(), context); + addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(context), context); +} + +DelayedCreatingSetsStep::DelayedCreatingSetsStep( + DataStream input_stream, PreparedSets::SubqueriesForSets subqueries_for_sets_, ContextPtr context_) + : subqueries_for_sets(std::move(subqueries_for_sets_)), context(std::move(context_)) +{ + input_streams = {input_stream}; + output_stream = std::move(input_stream); +} + +QueryPipelineBuilderPtr DelayedCreatingSetsStep::updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings &) +{ + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Cannot build pipeline in DelayedCreatingSets. This step should be optimized out."); } } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index b4777578a30..96ab26077fc 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -9,7 +9,7 @@ namespace DB { /// Creates sets for subqueries and JOIN. See CreatingSetsTransform. -class CreatingSetStep : public ITransformingStep, WithContext +class CreatingSetStep : public ITransformingStep { public: CreatingSetStep( @@ -32,6 +32,7 @@ private: String description; SubqueryForSet subquery_for_set; SizeLimits network_transfer_limits; + ContextPtr context; }; class CreatingSetsStep : public IQueryPlanStep @@ -46,6 +47,22 @@ public: void describePipeline(FormatSettings & settings) const override; }; +class DelayedCreatingSetsStep final : public IQueryPlanStep +{ +public: + DelayedCreatingSetsStep(DataStream input_stream, PreparedSets::SubqueriesForSets subqueries_for_sets_, ContextPtr context_); + + String getName() const override { return "DelayedCreatingSets"; } + + QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings &) override; + + static std::vector> makePlansForSets(DelayedCreatingSetsStep && step); + +private: + PreparedSets::SubqueriesForSets subqueries_for_sets; + ContextPtr context; +}; + void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context); void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index de1d43bed1b..2b934ec440b 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -110,6 +110,7 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &); bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes); bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes); +bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes); /// Enable memory bound merging of aggregation states for remote queries /// in case it was enabled for local plan diff --git a/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp new file mode 100644 index 00000000000..e9100ae9d02 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/addPlansForSets.cpp @@ -0,0 +1,35 @@ +#include +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes) +{ + auto * delayed = typeid_cast(node.step.get()); + if (!delayed) + return false; + + auto plans = DelayedCreatingSetsStep::makePlansForSets(std::move(*delayed)); + node.children.reserve(1 + plans.size()); + + DataStreams input_streams; + input_streams.reserve(1 + plans.size()); + input_streams.push_back(node.children.front()->step->getOutputStream()); + + for (const auto & plan : plans) + { + input_streams.push_back(plan->getCurrentDataStream()); + node.children.push_back(plan->getRootNode()); + nodes.splice(nodes.end(), QueryPlan::detachNodes(std::move(*plan))); + } + + auto creating_sets = std::make_unique(std::move(input_streams)); + creating_sets->setStepDescription("Create sets before main query execution"); + node.step = std::move(creating_sets); + return true; +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index 37e3b2f67d8..c74487f7cc1 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -163,6 +163,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s optimizePrewhere(stack, nodes); optimizePrimaryKeyCondition(stack); enableMemoryBoundMerging(*frame.node, nodes); + addPlansForSets(*frame.node, nodes); stack.pop_back(); } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 3fbe3d89845..cb732e58855 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -541,4 +541,9 @@ void QueryPlan::explainEstimate(MutableColumns & columns) } } +QueryPlan::Nodes QueryPlan::detachNodes(QueryPlan && plan) +{ + return std::move(plan.nodes); +} + } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 19d87b101de..d89bdc534be 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -105,10 +105,11 @@ public: std::vector children = {}; }; - const Node * getRootNode() const { return root; } - using Nodes = std::list; + Node * getRootNode() const { return root; } + static Nodes detachNodes(QueryPlan && plan); + private: QueryPlanResourceHolder resources; Nodes nodes; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index e3ae2d4fd4e..de10be599c8 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -76,7 +76,7 @@ void CreatingSetsTransform::startSubquery() } subquery.promise_to_fill_set.set_value(ready_set); - subquery.set_in_progress.reset(); + subquery.set.reset(); done_with_set = true; set_from_cache = true; } @@ -84,7 +84,7 @@ void CreatingSetsTransform::startSubquery() } } - if (subquery.set_in_progress) + if (subquery.set) LOG_TRACE(log, "Creating set, key: {}", subquery.key); if (subquery.table) LOG_TRACE(log, "Filling temporary table."); @@ -93,7 +93,7 @@ void CreatingSetsTransform::startSubquery() /// TODO: make via port table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); - done_with_set = !subquery.set_in_progress; + done_with_set = !subquery.set; done_with_table = !subquery.table; if ((done_with_set && !set_from_cache) /*&& done_with_join*/ && done_with_table) @@ -116,8 +116,8 @@ void CreatingSetsTransform::finishSubquery() } else if (read_rows != 0) { - if (subquery.set_in_progress) - LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set_in_progress->getTotalRowCount(), read_rows, seconds); + if (subquery.set) + LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set->getTotalRowCount(), read_rows, seconds); if (subquery.table) LOG_DEBUG(log, "Created Table with {} rows in {} sec.", read_rows, seconds); } @@ -131,9 +131,9 @@ void CreatingSetsTransform::init() { is_initialized = true; - if (subquery.set_in_progress) + if (subquery.set) { - subquery.set_in_progress->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); + subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); } watch.restart(); @@ -147,7 +147,7 @@ void CreatingSetsTransform::consume(Chunk chunk) if (!done_with_set) { - if (!subquery.set_in_progress->insertFromBlock(block.getColumnsWithTypeAndName())) + if (!subquery.set->insertFromBlock(block.getColumnsWithTypeAndName())) done_with_set = true; } @@ -170,12 +170,12 @@ void CreatingSetsTransform::consume(Chunk chunk) Chunk CreatingSetsTransform::generate() { - if (subquery.set_in_progress) + if (subquery.set) { - subquery.set_in_progress->finishInsert(); - subquery.promise_to_fill_set.set_value(subquery.set_in_progress); + subquery.set->finishInsert(); + subquery.promise_to_fill_set.set_value(subquery.set); if (promise_to_build) - promise_to_build->set_value(subquery.set_in_progress); + promise_to_build->set_value(subquery.set); } if (table_out.initialized()) diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index d5563108ad7..281236e631e 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -68,11 +68,18 @@ bool traverseASTFilter( PreparedSetKey set_key; if ((value->as() || value->as())) - set_key = PreparedSetKey::forSubquery(*value); + set_key = PreparedSetKey::forSubquery(value->getTreeHash()); else - set_key = PreparedSetKey::forLiteral(*value, {primary_key_type}); + set_key = PreparedSetKey::forLiteral(value->getTreeHash(), {primary_key_type}); - SetPtr set = prepared_sets->get(set_key); + FutureSetPtr future_set = prepared_sets->getFuture(set_key); + if (!future_set) + return false; + + if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); + + auto set = future_set->get(); if (!set) return false; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 5d961425469..7a1c3b10c8a 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1204,14 +1204,32 @@ bool KeyCondition::tryPrepareSetIndex( const auto right_arg = func.getArgumentAt(1); - auto prepared_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); + LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); + + auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); + if (!future_set) + return false; + + LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); + + if (!future_set->isReady()) + { + LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); + future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); + } + + auto prepared_set = future_set->get(); if (!prepared_set) return false; + LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); + /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) return false; + LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); + prepared_set->checkColumnsNumber(left_args_count); for (size_t i = 0; i < indexes_mapping.size(); ++i) prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]); diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 235d90bb974..5e186c25b83 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -310,7 +310,13 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (functionIsInOrGlobalInOperator(function_name)) { - ConstSetPtr prepared_set = rhs_argument.tryGetPreparedSet(); + auto future_set = rhs_argument.tryGetPreparedSet(); + if (future_set && !future_set->isReady()) + future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); + + ConstSetPtr prepared_set; + if (future_set) + prepared_set = future_set->get(); if (prepared_set && prepared_set->hasExplicitSetElements()) { diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 06fddd51cb8..5e1d23df3c7 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -624,7 +624,14 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( if (key_tuple_mapping.empty()) return false; - auto prepared_set = right_argument.tryGetPreparedSet(data_types); + auto future_set = right_argument.tryGetPreparedSet(data_types); + if (future_set && !future_set->isReady()) + future_set->buildOrderedSetInplace(right_argument.getTreeContext().getQueryContext()); + + ConstSetPtr prepared_set; + if (future_set) + prepared_set = future_set->get(); + if (!prepared_set || !prepared_set->hasExplicitSetElements()) return false; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index baa11368c8b..6ffba0ad029 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -655,7 +655,14 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( if (key_tuple_mapping.empty()) return false; - ConstSetPtr prepared_set = rhs.tryGetPreparedSet(); + auto future_set = rhs.tryGetPreparedSet(); + if (future_set && !future_set->isReady()) + future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); + + ConstSetPtr prepared_set; + if (future_set) + prepared_set = future_set->get(); + if (!prepared_set || !prepared_set->hasExplicitSetElements()) return false; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index e49459d3d17..e8843ff1489 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -275,7 +275,7 @@ bool RPNBuilderTreeNode::tryGetConstant(Field & output_value, DataTypePtr & outp namespace { -ConstSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) +FutureSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) { if (!dag_node->column) return {}; @@ -285,28 +285,20 @@ ConstSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) column = &column_const->getDataColumn(); if (const auto * column_set = typeid_cast(column)) - { - auto set = column_set->getData(); - - if (set && set->isCreated()) - return set; - } + return column_set->getData(); return {}; } } -ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const +FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const { const auto & prepared_sets = getTreeContext().getPreparedSets(); if (ast_node && prepared_sets) { - auto prepared_sets_with_same_hash = prepared_sets->getByTreeHash(ast_node->getTreeHash()); - for (auto & set : prepared_sets_with_same_hash) - if (set.isCreated()) - return set.get(); + return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); } else if (dag_node) { @@ -317,16 +309,16 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const return {}; } -ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) const +FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) const { const auto & prepared_sets = getTreeContext().getPreparedSets(); if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->get(PreparedSetKey::forSubquery(*ast_node)); + return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); - return prepared_sets->get(PreparedSetKey::forLiteral(*ast_node, data_types)); + return prepared_sets->getFuture(PreparedSetKey::forLiteral(ast_node->getTreeHash(), data_types)); } else if (dag_node) { @@ -337,7 +329,7 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) return nullptr; } -ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( +FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( const std::vector & indexes_mapping, const DataTypes & data_types) const { @@ -346,19 +338,25 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->get(PreparedSetKey::forSubquery(*ast_node)); + return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check /// that the types it was prepared with are compatible with the types of the primary key. - auto types_match = [&indexes_mapping, &data_types](const SetPtr & candidate_set) + auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) { assert(indexes_mapping.size() == data_types.size()); for (size_t i = 0; i < indexes_mapping.size(); ++i) { - if (!candidate_set->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i])) + if (indexes_mapping[i].tuple_index >= set_types.size()) + return false; + + auto lhs = recursiveRemoveLowCardinality(data_types[i]); + auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); + + if (!lhs->equals(*rhs)) return false; } @@ -366,10 +364,10 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( }; auto tree_hash = ast_node->getTreeHash(); - for (const auto & set : prepared_sets->getByTreeHash(tree_hash)) + for (const auto & [key, future_set] : prepared_sets->getSets()) { - if (set.isCreated() && types_match(set.get())) - return set.get(); + if (key.ast_hash == tree_hash && types_match(key.types)) + return future_set; } } else diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index 626eb288493..6f624d93cd6 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -109,13 +109,13 @@ public: bool tryGetConstant(Field & output_value, DataTypePtr & output_type) const; /// Try get prepared set from node - ConstSetPtr tryGetPreparedSet() const; + FutureSetPtr tryGetPreparedSet() const; /// Try get prepared set from node that match data types - ConstSetPtr tryGetPreparedSet(const DataTypes & data_types) const; + FutureSetPtr tryGetPreparedSet(const DataTypes & data_types) const; /// Try get prepared set from node that match indexes mapping and data types - ConstSetPtr tryGetPreparedSet( + FutureSetPtr tryGetPreparedSet( const std::vector & indexes_mapping, const DataTypes & data_types) const; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 6ca74406b17..0f45f6825f6 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -313,8 +313,15 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP if (!column_set) return; - auto set = column_set->getData(); - if (!set || !set->isCreated()) + auto future_set = column_set->getData(); + if (!future_set) + return; + + if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); + + auto set = future_set->get(); + if (!set) return; if (!set->hasExplicitSetElements()) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 4ff00facfdc..6e7b9ea5849 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -80,24 +80,24 @@ ASTPtr buildWhereExpression(const ASTs & functions) return makeASTFunction("and", functions); } -void buildSets(const ASTPtr & expression, ExpressionAnalyzer & analyzer) -{ - const auto * func = expression->as(); - if (func && functionIsInOrGlobalInOperator(func->name)) - { - const IAST & args = *func->arguments; - const ASTPtr & arg = args.children.at(1); - if (arg->as() || arg->as()) - { - analyzer.tryMakeSetForIndexFromSubquery(arg); - } - } - else - { - for (const auto & child : expression->children) - buildSets(child, analyzer); - } -} +// void buildSets(const ASTPtr & expression, ExpressionAnalyzer & analyzer) +// { +// const auto * func = expression->as(); +// if (func && functionIsInOrGlobalInOperator(func->name)) +// { +// const IAST & args = *func->arguments; +// const ASTPtr & arg = args.children.at(1); +// if (arg->as() || arg->as()) +// { +// analyzer.tryMakeSetForIndexFromSubquery(arg); +// } +// } +// else +// { +// for (const auto & child : expression->children) +// buildSets(child, analyzer); +// } +// } } @@ -199,7 +199,7 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex /// Let's analyze and calculate the prepared expression. auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(expression_ast, syntax_result, context); - buildSets(expression_ast, analyzer); + //buildSets(expression_ast, analyzer); ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes); Block block_with_filter = block; From 54ef6769f1131e49c87235c6948a39951adeae49 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 May 2023 18:30:08 +0000 Subject: [PATCH 004/171] Refactor PreparedSets [4] --- src/Interpreters/PreparedSets.cpp | 8 +- src/Interpreters/PreparedSets.h | 3 +- src/Planner/PlannerActionsVisitor.cpp | 20 +- .../optimizePrimaryKeyCondition.cpp | 2 + .../QueryPlan/ReadFromMergeTree.cpp | 173 ++++++++++++------ src/Processors/QueryPlan/ReadFromMergeTree.h | 11 +- .../QueryPlan/SourceStepWithFilter.h | 2 + .../Transforms/CreatingSetsTransform.cpp | 1 + src/Storages/MergeTree/KeyCondition.cpp | 10 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- 10 files changed, 160 insertions(+), 74 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 733eb1c24bb..5d9a0f27496 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -221,7 +221,7 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; - std::cerr << StackTrace().toString() << std::endl; + // std::cerr << StackTrace().toString() << std::endl; auto set_cache = context->getPreparedSetsCache(); if (set_cache) @@ -248,6 +248,10 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c auto plan = subquery.detachSource(); auto description = subquery.key; + // WriteBufferFromOwnString buf; + // plan->explainPlan(buf, {.header=true}); + // std::cerr << buf.str() << std::endl; + auto creating_set = std::make_unique( plan->getCurrentDataStream(), description, @@ -279,7 +283,7 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } -FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} +FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) { std::cerr << block.dumpStructure() << std::endl; } FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 8661d81a96a..b4d01754ea8 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -16,6 +16,7 @@ #include "Processors/Executors/CompletedPipelineExecutor.h" #include "Processors/QueryPlan/BuildQueryPipelineSettings.h" #include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" +#include "Processors/Sinks/EmptySink.h" #include "Processors/Sinks/NullSink.h" #include @@ -171,7 +172,7 @@ public: auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); + pipeline.complete(std::make_shared(Block())); CompletedPipelineExecutor executor(pipeline); executor.execute(); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 453b02a2f8f..e0844a6d2b1 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -629,14 +629,20 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma //auto set_key = planner_context->createSetKey(in_second_argument); - DataTypes set_element_types = {in_first_argument->getResultType()}; - const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); - if (left_tuple_type && left_tuple_type->getElements().size() != 1) - set_element_types = left_tuple_type->getElements(); + DataTypes set_element_types; - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); + auto in_second_argument_node_type = in_second_argument->getNodeType(); + if (!(in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION)) + { + set_element_types = {in_first_argument->getResultType()}; + const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); + if (left_tuple_type && left_tuple_type->getElements().size() != 1) + set_element_types = left_tuple_type->getElements(); + + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + } auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index e98386a6ee9..5ef786ff975 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -28,6 +28,8 @@ void optimizePrimaryKeyCondition(const Stack & stack) else break; } + + source_step_with_filter->onAddFilterFinish(); } } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 185ec9bace8..ad63b486c7c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -247,7 +247,7 @@ ReadFromMergeTree::ReadFromMergeTree( { /// build sort description for output stream SortDescription sort_description; - const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(); + const Names & sorting_key_columns = metadata_for_reading->getSortingKeyColumns(); const Block & header = output_stream->header; const int sort_direction = getSortDirection(); for (const auto & column_name : sorting_key_columns) @@ -1118,7 +1118,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge prewhere_info, filter_nodes, storage_snapshot->metadata, - storage_snapshot->getMetadataForQuery(), + metadata_for_reading, query_info, context, requested_num_streams, @@ -1126,7 +1126,90 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); +} + +static ActionsDAGPtr buildFilterDAG( + const ContextPtr & context, + const PrewhereInfoPtr & prewhere_info, + const ActionDAGNodes & added_filter_nodes, + const SelectQueryInfo & query_info) +{ + const auto & settings = context->getSettingsRef(); + ActionsDAG::NodeRawConstPtrs nodes; + + if (prewhere_info) + { + { + const auto & node = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); + nodes.push_back(&node); + } + + if (prewhere_info->row_level_filter) + { + const auto & node = prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name); + nodes.push_back(&node); + } + } + + for (const auto & node : added_filter_nodes.nodes) + nodes.push_back(node); + + std::unordered_map node_name_to_input_node_column; + + if (settings.allow_experimental_analyzer && query_info.planner_context) + { + const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression); + for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) + { + const auto & column = table_expression_data.getColumnOrThrow(column_name); + node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name)); + } + } + + return ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); +} + +static void buildKeyCondition( + std::optional & key_condition, + ActionsDAGPtr filter_actions_dag, + const ContextPtr & context, + const SelectQueryInfo & query_info, + const StorageMetadataPtr & metadata_snapshot) +{ + key_condition.reset(); + + // Build and check if primary key is used when necessary + const auto & primary_key = metadata_snapshot->getPrimaryKey(); + const Names & primary_key_column_names = primary_key.column_names; + + const auto & settings = context->getSettingsRef(); + if (settings.query_plan_optimize_primary_key) + { + NameSet array_join_name_set; + if (query_info.syntax_analyzer_result) + array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); + + key_condition.emplace(filter_actions_dag, + context, + primary_key_column_names, + primary_key.expression, + array_join_name_set); + } + else + { + key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); + } +} + +void ReadFromMergeTree::onAddFilterFinish() +{ + if (!filter_nodes.nodes.empty()) + { + auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); + buildKeyCondition(key_condition, filter_actions_dag, context, query_info, metadata_for_reading); + } } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( @@ -1142,44 +1225,14 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log) + Poco::Logger * log, + std::optional & key_condition) { const auto & settings = context->getSettingsRef(); if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key) { - ActionsDAG::NodeRawConstPtrs nodes; - - if (prewhere_info) - { - { - const auto & node = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name); - nodes.push_back(&node); - } - - if (prewhere_info->row_level_filter) - { - const auto & node = prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name); - nodes.push_back(&node); - } - } - - for (const auto & node : added_filter_nodes.nodes) - nodes.push_back(node); - - std::unordered_map node_name_to_input_node_column; - - if (settings.allow_experimental_analyzer && query_info.planner_context) - { - const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression); - for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName()) - { - const auto & column = table_expression_data.getColumnOrThrow(column_name); - node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name)); - } - } - auto updated_query_info_with_filter_dag = query_info; - updated_query_info_with_filter_dag.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); + updated_query_info_with_filter_dag.filter_actions_dag = buildFilterDAG(context, prewhere_info, added_filter_nodes, query_info); return selectRangesToReadImpl( parts, @@ -1192,7 +1245,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); } return selectRangesToReadImpl( @@ -1206,7 +1260,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( @@ -1220,7 +1275,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log) + Poco::Logger * log, + std::optional & key_condition) { AnalysisResult result; const auto & settings = context->getSettingsRef(); @@ -1246,24 +1302,29 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; - std::optional key_condition; - if (settings.query_plan_optimize_primary_key) - { - NameSet array_join_name_set; - if (query_info.syntax_analyzer_result) - array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); + // if (!key_condition) + // { + // if (settings.query_plan_optimize_primary_key) + // { + // NameSet array_join_name_set; + // if (query_info.syntax_analyzer_result) + // array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); - key_condition.emplace(query_info.filter_actions_dag, - context, - primary_key_column_names, - primary_key.expression, - array_join_name_set); - } - else - { - key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); - } + // key_condition.emplace(query_info.filter_actions_dag, + // context, + // primary_key_column_names, + // primary_key.expression, + // array_join_name_set); + // } + // else + // { + // key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); + // } + // } + + if (!key_condition) + buildKeyCondition(key_condition, query_info.filter_actions_dag, context, query_info, metadata_snapshot); if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue()) { @@ -1395,7 +1456,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, /// update sort info for output stream SortDescription sort_description; - const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(); + const Names & sorting_key_columns = metadata_for_reading->getSortingKeyColumns(); const Block & header = output_stream->header; const int sort_direction = getSortDirection(); for (const auto & column_name : sorting_key_columns) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 5e4ba117967..121970f2ca9 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -145,7 +145,8 @@ public: const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log); + Poco::Logger * log, + std::optional & key_condition); MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; @@ -177,6 +178,8 @@ public: size_t getNumStreams() const { return requested_num_streams; } bool isParallelReadingEnabled() const { return read_task_callback != std::nullopt; } + void onAddFilterFinish() override; + private: static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl( MergeTreeData::DataPartsVector parts, @@ -189,7 +192,8 @@ private: const MergeTreeData & data, const Names & real_column_names, bool sample_factor_column_queried, - Poco::Logger * log); + Poco::Logger * log, + std::optional & key_condition); int getSortDirection() const { @@ -228,6 +232,9 @@ private: std::shared_ptr max_block_numbers_to_read; + /// Pre-computed value, needed to trigger sets creatin for PK + mutable std::optional key_condition; + Poco::Logger * log; UInt64 selected_parts = 0; UInt64 selected_rows = 0; diff --git a/src/Processors/QueryPlan/SourceStepWithFilter.h b/src/Processors/QueryPlan/SourceStepWithFilter.h index a363451fff2..34b6e3c6a7b 100644 --- a/src/Processors/QueryPlan/SourceStepWithFilter.h +++ b/src/Processors/QueryPlan/SourceStepWithFilter.h @@ -37,6 +37,8 @@ public: filter_dags.push_back(std::move(filter_dag)); } + virtual void onAddFilterFinish() {} + protected: std::vector filter_dags; ActionDAGNodes filter_nodes; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index de10be599c8..6626d4b9795 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -133,6 +133,7 @@ void CreatingSetsTransform::init() if (subquery.set) { + //std::cerr << "=========== " << getInputPort().getHeader().dumpStructure() << std::endl; subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7a1c3b10c8a..efeb9e40dd4 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1204,17 +1204,17 @@ bool KeyCondition::tryPrepareSetIndex( const auto right_arg = func.getArgumentAt(1); - LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); if (!future_set) return false; - LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); if (!future_set->isReady()) { - LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); } @@ -1222,13 +1222,13 @@ bool KeyCondition::tryPrepareSetIndex( if (!prepared_set) return false; - LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) return false; - LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); + // LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); prepared_set->checkColumnsNumber(left_args_count); for (size_t i = 0; i < indexes_mapping.size(); ++i) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 22df8f298c4..aae8f843a3c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1291,6 +1291,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); + std::optional key_condition; return ReadFromMergeTree::selectRangesToRead( std::move(parts), prewhere_info, @@ -1304,7 +1305,8 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar data, real_column_names, sample_factor_column_queried, - log); + log, + key_condition); } QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( From b2c36fc3e5a968cd3223261585cb00f89be2c783 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 5 May 2023 20:51:38 +0000 Subject: [PATCH 005/171] Fixing style. --- src/Interpreters/PreparedSets.cpp | 7 ++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 5d9a0f27496..1d7d90432b0 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -11,6 +11,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) { /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, @@ -283,7 +288,7 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } -FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) { std::cerr << block.dumpStructure() << std::endl; } +FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 121970f2ca9..f13f75bfebc 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -232,7 +232,7 @@ private: std::shared_ptr max_block_numbers_to_read; - /// Pre-computed value, needed to trigger sets creatin for PK + /// Pre-computed value, needed to trigger sets creating for PK mutable std::optional key_condition; Poco::Logger * log; From fc02e9efc9eb2dcb9b7209e41eafb1e50abced7d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 8 May 2023 17:23:46 +0000 Subject: [PATCH 006/171] update fasttest a bit --- tests/ci/fast_test_check.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 89066ade2cb..fe211d79810 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -214,8 +214,11 @@ def main(): # Refuse other checks to run if fast test failed if state != "success": - if FORCE_TESTS_LABEL in pr_info.labels and state != "error": - print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") + if state == "error": + print("The status is 'error', report failure disregard the labels") + sys.exit(1) + elif FORCE_TESTS_LABEL in pr_info.labels: + print(f"'{FORCE_TESTS_LABEL}' enabled, reporting success") else: sys.exit(1) From 58bdcc29315a712e1255c13d31669f4545de9edb Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 10 May 2023 23:55:13 +0000 Subject: [PATCH 007/171] allow to cast IPv6 to IPv4 for address in proper mapping block --- src/Functions/FunctionsConversion.h | 30 ++++++++++++++++++++++++++--- 1 file changed, 27 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9c4085f9745..9cdd09780e3 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -57,6 +57,7 @@ #include #include #include +#include #include @@ -217,13 +218,13 @@ struct ConvertImpl } else if constexpr ( (std::is_same_v != std::is_same_v) - && !(is_any_of || is_any_of) + && !(is_any_of || is_any_of) ) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion from {} to {} is not supported", TypeName, TypeName); } - else if constexpr (std::is_same_v != std::is_same_v) + else if constexpr (std::is_same_v != std::is_same_v && !std::is_same_v) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion between numeric types and IPv6 is not supported. " @@ -304,7 +305,30 @@ struct ConvertImpl } else { - if constexpr (std::is_same_v && std::is_same_v) + if constexpr (std::is_same_v && std::is_same_v) + { + const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; + const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); + if (!matchIPv6Subnet(src, ip4_cidr, 96)) + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", named_from.column->getName()); + + uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); + if constexpr (std::endian::native == std::endian::little) + { + dst[0] = src[15]; + dst[1] = src[14]; + dst[2] = src[13]; + dst[3] = src[12]; + } + else + { + dst[3] = src[15]; + dst[2] = src[14]; + dst[1] = src[13]; + dst[0] = src[12]; + } + } + else if constexpr (std::is_same_v && std::is_same_v) vec_to[i] = static_cast(static_cast(vec_from[i])); else vec_to[i] = static_cast(vec_from[i]); From 108e256578574b26f8adeb3916b15238f0557ee9 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 11 May 2023 16:17:52 +0000 Subject: [PATCH 008/171] allow to cast IPv4 to IPv6 --- src/Functions/FunctionsConversion.h | 32 ++++++++++++++++++++++++----- 1 file changed, 27 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 9cdd09780e3..5bf59f33cb5 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -218,13 +218,13 @@ struct ConvertImpl } else if constexpr ( (std::is_same_v != std::is_same_v) - && !(is_any_of || is_any_of) + && !(is_any_of || is_any_of) ) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion from {} to {} is not supported", TypeName, TypeName); } - else if constexpr (std::is_same_v != std::is_same_v && !std::is_same_v) + else if constexpr (std::is_same_v != std::is_same_v && !(std::is_same_v || std::is_same_v)) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Conversion between numeric types and IPv6 is not supported. " @@ -322,10 +322,32 @@ struct ConvertImpl } else { - dst[3] = src[15]; - dst[2] = src[14]; - dst[1] = src[13]; dst[0] = src[12]; + dst[1] = src[13]; + dst[2] = src[14]; + dst[3] = src[15]; + } + } + else if constexpr (std::is_same_v && std::is_same_v) + { + const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); + uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); + std::memset(dst, '\0', IPV6_BINARY_LENGTH); + dst[10] = dst[11] = 0xff; + + if constexpr (std::endian::native == std::endian::little) + { + dst[12] = src[3]; + dst[13] = src[2]; + dst[14] = src[1]; + dst[15] = src[0]; + } + else + { + dst[12] = src[0]; + dst[13] = src[1]; + dst[14] = src[2]; + dst[15] = src[3]; } } else if constexpr (std::is_same_v && std::is_same_v) From 24067ea977b6e4484f68efba8858ba8d0ad1cd6b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 12 May 2023 15:54:50 +0000 Subject: [PATCH 009/171] allow conversion for toIPv4OrDefault --- src/Functions/FunctionsCodingIP.h | 81 +++++++++++++++++++++++++++++ src/Functions/FunctionsConversion.h | 20 +++++++ 2 files changed, 101 insertions(+) diff --git a/src/Functions/FunctionsCodingIP.h b/src/Functions/FunctionsCodingIP.h index d02cc81f608..bd53fa7e043 100644 --- a/src/Functions/FunctionsCodingIP.h +++ b/src/Functions/FunctionsCodingIP.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -16,6 +17,7 @@ namespace ErrorCodes extern const int CANNOT_PARSE_IPV4; extern const int CANNOT_PARSE_IPV6; extern const int ILLEGAL_COLUMN; + extern const int CANNOT_CONVERT_TYPE; } enum class IPStringToNumExceptionMode : uint8_t @@ -296,4 +298,83 @@ ColumnPtr convertToIPv4(ColumnPtr column, const PaddedPODArray * null_map return col_res; } +template +ColumnPtr convertIPv6ToIPv4(ColumnPtr column, const PaddedPODArray * null_map = nullptr) +{ + const ColumnIPv6 * column_ipv6 = checkAndGetColumn(column.get()); + + if (!column_ipv6) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column type {}. Expected IPv6.", column->getName()); + + size_t column_size = column_ipv6->size(); + + ColumnUInt8::MutablePtr col_null_map_to; + ColumnUInt8::Container * vec_null_map_to = nullptr; + + if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + { + col_null_map_to = ColumnUInt8::create(column_size, false); + vec_null_map_to = &col_null_map_to->getData(); + } + + const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; + + auto col_res = ToColumn::create(); + auto & vec_res = col_res->getData(); + vec_res.resize(column_size); + const auto & vec_src = column_ipv6->getData(); + + for (size_t i = 0; i < vec_res.size(); ++i) + { + const uint8_t * src = reinterpret_cast(&vec_src[i]); + uint8_t * dst = reinterpret_cast(&vec_res[i]); + + if (null_map && (*null_map)[i]) + { + std::memset(dst, '\0', IPV4_BINARY_LENGTH); + if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + (*vec_null_map_to)[i] = true; + continue; + } + + if (!matchIPv6Subnet(src, ip4_cidr, 96)) + { + if constexpr (exception_mode == IPStringToNumExceptionMode::Throw) + { + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", column->getName()); + } + else if constexpr (exception_mode == IPStringToNumExceptionMode::Default) + { + std::memset(dst, '\0', IPV4_BINARY_LENGTH); + } + else if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + { + (*vec_null_map_to)[i] = true; + std::memset(dst, '\0', IPV4_BINARY_LENGTH); + } + continue; + } + + if constexpr (std::endian::native == std::endian::little) + { + dst[0] = src[15]; + dst[1] = src[14]; + dst[2] = src[13]; + dst[3] = src[12]; + } + else + { + dst[0] = src[12]; + dst[1] = src[13]; + dst[2] = src[14]; + dst[3] = src[15]; + } + } + + if constexpr (exception_mode == IPStringToNumExceptionMode::Null) + return ColumnNullable::create(std::move(col_res), std::move(col_null_map_to)); + + return col_res; +} + } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 5bf59f33cb5..4d4efc84df1 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -4035,6 +4035,26 @@ private: return true; } } + else if constexpr (WhichDataType(FromDataType::type_id).isIPv6() && WhichDataType(ToDataType::type_id).isIPv4()) + { + ret = [cast_ipv4_ipv6_default_on_conversion_error_value, requested_result_is_nullable]( + ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * column_nullable, size_t) + -> ColumnPtr + { + if (!WhichDataType(result_type).isIPv4()) + throw Exception( + ErrorCodes::TYPE_MISMATCH, "Wrong result type {}. Expected IPv4", result_type->getName()); + + const auto * null_map = column_nullable ? &column_nullable->getNullMapData() : nullptr; + if (cast_ipv4_ipv6_default_on_conversion_error_value || requested_result_is_nullable) + return convertIPv6ToIPv4(arguments[0].column, null_map); + else + return convertIPv6ToIPv4(arguments[0].column, null_map); + }; + + return true; + } + if constexpr (WhichDataType(ToDataType::type_id).isStringOrFixedString()) { if (from_type->getCustomSerialization()) From fc857aa2dbf297d2681af16ddfbafb47739db854 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 15 May 2023 03:06:03 +0000 Subject: [PATCH 010/171] tests added --- .../queries/0_stateless/02234_cast_to_ip_address.reference | 4 ++++ tests/queries/0_stateless/02234_cast_to_ip_address.sql | 7 +++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/02234_cast_to_ip_address.reference b/tests/queries/0_stateless/02234_cast_to_ip_address.reference index 9023b36a9bf..fa9c6bd0f94 100644 --- a/tests/queries/0_stateless/02234_cast_to_ip_address.reference +++ b/tests/queries/0_stateless/02234_cast_to_ip_address.reference @@ -11,6 +11,10 @@ IPv4 functions 127.0.0.1 127.0.0.1 -- +1.2.3.4 +1.2.3.4 +0.0.0.0 +-- 127.0.0.1 -- 0 diff --git a/tests/queries/0_stateless/02234_cast_to_ip_address.sql b/tests/queries/0_stateless/02234_cast_to_ip_address.sql index 6c65fe86cc9..28f1afff57f 100644 --- a/tests/queries/0_stateless/02234_cast_to_ip_address.sql +++ b/tests/queries/0_stateless/02234_cast_to_ip_address.sql @@ -20,6 +20,13 @@ SELECT toIPv4OrNull('127.0.0.1'); SELECT '--'; +SELECT toIPv4(toIPv6('::ffff:1.2.3.4')); +SELECT toIPv4(toIPv6('::afff:1.2.3.4')); --{serverError CANNOT_CONVERT_TYPE} +SELECT toIPv4OrDefault(toIPv6('::ffff:1.2.3.4')); +SELECT toIPv4OrDefault(toIPv6('::afff:1.2.3.4')); + +SELECT '--'; + SELECT cast('test' , 'IPv4'); --{serverError CANNOT_PARSE_IPV4} SELECT cast('127.0.0.1' , 'IPv4'); From 6de52e9fced5c71c24a2f40a2b13b9c3d5656b14 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 May 2023 18:36:02 +0000 Subject: [PATCH 011/171] Fixing some tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 15 ++- src/Interpreters/PreparedSets.cpp | 36 ++++- src/Interpreters/PreparedSets.h | 32 +---- src/Planner/Planner.cpp | 15 ++- src/Processors/QueryPlan/CreatingSetsStep.h | 3 + .../Optimizations/filterPushDown.cpp | 13 ++ .../QueryPlan/ReadFromMergeTree.cpp | 107 +++++++++------ src/Processors/QueryPlan/ReadFromMergeTree.h | 44 +++++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 127 ++++++------------ .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- .../MergeTreeIndexConditionBloomFilter.cpp | 10 ++ .../MergeTree/MergeTreeIndexInverted.cpp | 5 + src/Storages/MergeTree/RPNBuilder.cpp | 20 ++- src/Storages/SelectQueryInfo.h | 2 + 14 files changed, 259 insertions(+), 172 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index bd96ba693fe..8d305c07ce9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3088,12 +3088,17 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - auto step = std::make_unique( - query_plan.getCurrentDataStream(), - prepared_sets->detachSubqueries(context), - context); + auto subqueries = prepared_sets->detachSubqueries(context); - query_plan.addStep(std::move(step)); + if (!subqueries.empty()) + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + std::move(subqueries), + context); + + query_plan.addStep(std::move(step)); + } } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 1d7d90432b0..cd6b2a81ba0 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -226,7 +226,7 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; - // std::cerr << StackTrace().toString() << std::endl; + //std::cerr << StackTrace().toString() << std::endl; auto set_cache = context->getPreparedSetsCache(); if (set_cache) @@ -294,4 +294,38 @@ FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquer FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} +SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, true); + fill(size_limits, settings.transform_null_in, true); + return set; +} + +std::unique_ptr FutureSetFromTuple::build(const ContextPtr & context) +{ + const auto & settings = context->getSettingsRef(); + auto size_limits = getSizeLimitsForSet(settings, false); + fill(size_limits, settings.transform_null_in, false); + return nullptr; +} + +void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) +{ + fill(size_limits, transform_null_in, false); +} + +void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) +{ + //std::cerr << StackTrace().toString() << std::endl; + + if (set) + return; + + set = std::make_shared(size_limits, create_ordered_set, transform_null_in); + set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); + set->insertFromBlock(block.getColumnsWithTypeAndName()); + set->finishInsert(); +} + }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index b4d01754ea8..ef7aba38f24 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -86,42 +86,18 @@ public: bool isFilled() const override { return true; } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr & context) override - { - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, true); - fill(size_limits, settings.transform_null_in, true); - return set; - } + SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - std::unique_ptr build(const ContextPtr & context) override - { - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, false); - fill(size_limits, settings.transform_null_in, false); - return nullptr; - } + std::unique_ptr build(const ContextPtr & context) override; - void buildForTuple(SizeLimits size_limits, bool transform_null_in) - { - fill(size_limits, transform_null_in, false); - } + void buildForTuple(SizeLimits size_limits, bool transform_null_in); private: Block block; SetPtr set; - void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) - { - if (set) - return; - - set = std::make_shared(size_limits, create_ordered_set, transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(block.getColumnsWithTypeAndName()); - set->finishInsert(); - } + void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); }; /// Information on how to build set for the [GLOBAL] IN section. diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 38d0aa29d24..b1780212e51 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1468,12 +1468,17 @@ void Planner::buildPlanForQueryNode() if (!select_query_options.only_analyze) { - auto step = std::make_unique( - query_plan.getCurrentDataStream(), - planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), - planner_context->getQueryContext()); + auto subqueries = planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()); - query_plan.addStep(std::move(step)); + if (!subqueries.empty()) + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + std::move(subqueries), + planner_context->getQueryContext()); + + query_plan.addStep(std::move(step)); + } //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); //addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 96ab26077fc..244bb27ba78 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -58,6 +58,9 @@ public: static std::vector> makePlansForSets(DelayedCreatingSetsStep && step); + ContextPtr getContext() const { return context; } + PreparedSets::SubqueriesForSets detachSubqueries() { return std::move(subqueries_for_sets); } + private: PreparedSets::SubqueriesForSets subqueries_for_sets; ContextPtr context; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 37bc894339f..63ba3d5b56c 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -268,6 +268,19 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return 2; } + if (auto * delayed = typeid_cast(child.get())) + { + /// CreatingSets does not change header. + /// We can push down filter and update header. + /// Filter - DelayedCreatingSets - Something + child = std::make_unique(filter->getOutputStream(), delayed->detachSubqueries(), delayed->getContext()); + std::swap(parent, child); + std::swap(parent_node->children, child_node->children); + std::swap(parent_node->children.front(), child_node->children.front()); + /// DelayedCreatingSets - Filter - Something + return 2; + } + if (auto * totals_having = typeid_cast(child.get())) { /// If totals step has HAVING expression, skip it for now. diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index ef013a36069..a096538a298 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1127,7 +1127,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } static ActionsDAGPtr buildFilterDAG( @@ -1171,14 +1171,14 @@ static ActionsDAGPtr buildFilterDAG( return ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context); } -static void buildKeyCondition( - std::optional & key_condition, +static void buildIndexes( + std::optional & indexes, ActionsDAGPtr filter_actions_dag, const ContextPtr & context, const SelectQueryInfo & query_info, const StorageMetadataPtr & metadata_snapshot) { - key_condition.reset(); + indexes.reset(); // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); @@ -1191,16 +1191,58 @@ static void buildKeyCondition( if (query_info.syntax_analyzer_result) array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); - key_condition.emplace(filter_actions_dag, + indexes.emplace(ReadFromMergeTree::Indexes{{ + filter_actions_dag, context, primary_key_column_names, primary_key.expression, - array_join_name_set); + array_join_name_set}, {}, false}); } else { - key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); + indexes.emplace(ReadFromMergeTree::Indexes{{ + query_info, + context, + primary_key_column_names, + primary_key.expression}, {}, false}); } + + indexes->use_skip_indexes = settings.use_skip_indexes; + bool final = query_info.isFinal(); + + if (final && !settings.use_skip_indexes_if_final) + indexes->use_skip_indexes = false; + + if (!indexes->use_skip_indexes) + return; + + UsefulSkipIndexes skip_indexes; + using Key = std::pair; + std::map merged; + + for (const auto & index : metadata_snapshot->getSecondaryIndices()) + { + auto index_helper = MergeTreeIndexFactory::instance().get(index); + if (index_helper->isMergeable()) + { + auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); + if (inserted) + { + skip_indexes.merged_indices.emplace_back(); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + } + + skip_indexes.merged_indices[it->second].addIndex(index_helper); + } + else + { + auto condition = index_helper->createIndexCondition(query_info, context); + if (!condition->alwaysUnknownOrTrue()) + skip_indexes.useful_indices.emplace_back(index_helper, condition); + } + } + + indexes->skip_indexes = std::move(skip_indexes); } void ReadFromMergeTree::onAddFilterFinish() @@ -1208,7 +1250,7 @@ void ReadFromMergeTree::onAddFilterFinish() if (!filter_nodes.nodes.empty()) { auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); - buildKeyCondition(key_condition, filter_actions_dag, context, query_info, metadata_for_reading); + buildIndexes(indexes, filter_actions_dag, context, query_info, metadata_for_reading); } } @@ -1226,7 +1268,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition) + std::optional & indexes) { const auto & settings = context->getSettingsRef(); if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key) @@ -1246,7 +1288,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } return selectRangesToReadImpl( @@ -1261,7 +1303,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( @@ -1276,7 +1318,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition) + std::optional & indexes) { AnalysisResult result; const auto & settings = context->getSettingsRef(); @@ -1323,10 +1365,10 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( // } // } - if (!key_condition) - buildKeyCondition(key_condition, query_info.filter_actions_dag, context, query_info, metadata_snapshot); + if (!indexes) + buildIndexes(indexes, query_info.filter_actions_dag, context, query_info, metadata_snapshot); - if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue()) + if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue()) { return std::make_shared(MergeTreeDataSelectAnalysisResult{ .result = std::make_exception_ptr(Exception( @@ -1334,9 +1376,9 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( "Primary key ({}) is not used and setting 'force_primary_key' is set", fmt::join(primary_key_column_names, ", ")))}); } - LOG_DEBUG(log, "Key condition: {}", key_condition->toString()); + LOG_DEBUG(log, "Key condition: {}", indexes->key_condition.toString()); - if (key_condition->alwaysFalse()) + if (indexes->key_condition.alwaysFalse()) return std::make_shared(MergeTreeDataSelectAnalysisResult{.result = std::move(result)}); size_t total_marks_pk = 0; @@ -1358,7 +1400,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( query_info, metadata_snapshot->getColumns().getAllPhysical(), parts, - *key_condition, + indexes->key_condition, data, metadata_snapshot, context, @@ -1374,23 +1416,17 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( auto reader_settings = getMergeTreeReaderSettings(context, query_info); - bool use_skip_indexes = settings.use_skip_indexes; - bool final = isFinal(query_info); - - if (final && !settings.use_skip_indexes_if_final) - use_skip_indexes = false; - result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( std::move(parts), metadata_snapshot, - query_info, context, - *key_condition, + indexes->key_condition, + indexes->skip_indexes, reader_settings, log, num_streams, result.index_stats, - use_skip_indexes); + indexes->use_skip_indexes); } catch (...) { @@ -1438,7 +1474,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, /// Disable read-in-order optimization for reverse order with final. /// Otherwise, it can lead to incorrect final behavior because the implementation may rely on the reading in direct order). - if (direction != 1 && isFinal(query_info)) + if (direction != 1 && query_info.isFinal()) return false; auto order_info = std::make_shared(SortDescription{}, prefix_size, direction, limit); @@ -1561,11 +1597,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const bool ReadFromMergeTree::isQueryWithFinal() const { - const auto & select = query_info.query->as(); - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - else - return select.final(); + return query_info.isFinal(); } bool ReadFromMergeTree::isQueryWithSampling() const @@ -2009,15 +2041,6 @@ void ReadFromMergeTree::describeIndexes(JSONBuilder::JSONMap & map) const } } -bool ReadFromMergeTree::isFinal(const SelectQueryInfo & query_info) -{ - if (query_info.table_expression_modifiers) - return query_info.table_expression_modifiers->hasFinal(); - - const auto & select = query_info.query->as(); - return select.final(); -} - bool MergeTreeDataSelectAnalysisResult::error() const { return std::holds_alternative(result); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index f13f75bfebc..6610b463726 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -24,6 +24,35 @@ struct MergeTreeDataSelectSamplingData ActionsDAGPtr filter_expression; }; +struct UsefulSkipIndexes +{ + struct DataSkippingIndexAndCondition + { + MergeTreeIndexPtr index; + MergeTreeIndexConditionPtr condition; + + DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) + : index(index_), condition(condition_) + { + } + }; + + struct MergedDataSkippingIndexAndCondition + { + std::vector indices; + MergeTreeIndexMergedConditionPtr condition; + + void addIndex(const MergeTreeIndexPtr & index) + { + indices.push_back(index); + condition->addIndex(indices.back()); + } + }; + + std::vector useful_indices; + std::vector merged_indices; +}; + struct MergeTreeDataSelectAnalysisResult; using MergeTreeDataSelectAnalysisResultPtr = std::shared_ptr; @@ -132,6 +161,13 @@ public: UInt64 getSelectedRows() const { return selected_rows; } UInt64 getSelectedMarks() const { return selected_marks; } + struct Indexes + { + KeyCondition key_condition; + UsefulSkipIndexes skip_indexes; + bool use_skip_indexes; + }; + static MergeTreeDataSelectAnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, const PrewhereInfoPtr & prewhere_info, @@ -146,7 +182,7 @@ public: const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition); + std::optional & indexes); MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; @@ -159,8 +195,6 @@ public: bool requestReadingInOrder(size_t prefix_size, int direction, size_t limit); void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value); - - static bool isFinal(const SelectQueryInfo & query_info); bool isQueryWithFinal() const; bool isQueryWithSampling() const; @@ -193,7 +227,7 @@ private: const Names & real_column_names, bool sample_factor_column_queried, Poco::Logger * log, - std::optional & key_condition); + std::optional & indexes); int getSortDirection() const { @@ -233,7 +267,7 @@ private: std::shared_ptr max_block_numbers_to_read; /// Pre-computed value, needed to trigger sets creating for PK - mutable std::optional key_condition; + mutable std::optional indexes; Poco::Logger * log; UInt64 selected_parts = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 55dfc56d8ac..f99e15c0fc1 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -887,9 +887,9 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, - const SelectQueryInfo & query_info, const ContextPtr & context, const KeyCondition & key_condition, + const UsefulSkipIndexes & skip_indexes, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, @@ -900,66 +900,6 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd parts_with_ranges.resize(parts.size()); const Settings & settings = context->getSettingsRef(); - /// Let's start analyzing all useful indices - - struct IndexStat - { - std::atomic total_granules{0}; - std::atomic granules_dropped{0}; - std::atomic total_parts{0}; - std::atomic parts_dropped{0}; - }; - - struct DataSkippingIndexAndCondition - { - MergeTreeIndexPtr index; - MergeTreeIndexConditionPtr condition; - IndexStat stat; - - DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) - : index(index_), condition(condition_) - { - } - }; - - struct MergedDataSkippingIndexAndCondition - { - std::vector indices; - MergeTreeIndexMergedConditionPtr condition; - IndexStat stat; - - void addIndex(const MergeTreeIndexPtr & index) - { - indices.push_back(index); - condition->addIndex(indices.back()); - } - }; - - std::list useful_indices; - std::map, MergedDataSkippingIndexAndCondition> merged_indices; - - if (use_skip_indexes) - { - for (const auto & index : metadata_snapshot->getSecondaryIndices()) - { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) - { - auto [it, inserted] = merged_indices.try_emplace({index_helper->index.type, index_helper->getGranularity()}); - if (inserted) - it->second.condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); - - it->second.addIndex(index_helper); - } - else - { - auto condition = index_helper->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - useful_indices.emplace_back(index_helper, condition); - } - } - } - if (use_skip_indexes && settings.force_data_skipping_indices.changed) { const auto & indices = settings.force_data_skipping_indices.toString(); @@ -977,7 +917,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices); std::unordered_set useful_indices_names; - for (const auto & useful_index : useful_indices) + for (const auto & useful_index : skip_indexes.useful_indices) useful_indices_names.insert(useful_index.index->index.name); for (const auto & index_name : forced_indices) @@ -992,6 +932,17 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd } } + struct IndexStat + { + std::atomic total_granules{0}; + std::atomic granules_dropped{0}; + std::atomic total_parts{0}; + std::atomic parts_dropped{0}; + }; + + std::vector useful_indices_stat(skip_indexes.useful_indices.size()); + std::vector merged_indices_stat(skip_indexes.merged_indices.size()); + std::atomic sum_marks_pk = 0; std::atomic sum_parts_pk = 0; @@ -1018,12 +969,14 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (!ranges.ranges.empty()) sum_parts_pk.fetch_add(1, std::memory_order_relaxed); - for (auto & index_and_condition : useful_indices) + for (size_t idx = 0; idx < skip_indexes.useful_indices.size(); ++idx) { if (ranges.ranges.empty()) break; - index_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed); + auto & index_and_condition = skip_indexes.useful_indices[idx]; + auto & stat = useful_indices_stat[idx]; + stat.total_parts.fetch_add(1, std::memory_order_relaxed); size_t total_granules = 0; size_t granules_dropped = 0; @@ -1040,19 +993,21 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd uncompressed_cache.get(), log); - index_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); - index_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); + stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); if (ranges.ranges.empty()) - index_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); + stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); } - for (auto & [_, indices_and_condition] : merged_indices) + for (size_t idx = 0; idx < skip_indexes.merged_indices.size(); ++idx) { if (ranges.ranges.empty()) break; - indices_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed); + auto & indices_and_condition = skip_indexes.merged_indices[idx]; + auto & stat = merged_indices_stat[idx]; + stat.total_parts.fetch_add(1, std::memory_order_relaxed); size_t total_granules = 0; size_t granules_dropped = 0; @@ -1063,11 +1018,11 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd total_granules, granules_dropped, mark_cache.get(), uncompressed_cache.get(), log); - indices_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); - indices_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); + stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); if (ranges.ranges.empty()) - indices_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); + stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); } if (!ranges.ranges.empty()) @@ -1134,15 +1089,17 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd .num_granules_after = sum_marks_pk.load(std::memory_order_relaxed)}); } - for (const auto & index_and_condition : useful_indices) + for (size_t idx = 0; idx < skip_indexes.useful_indices.size(); ++idx) { + const auto & index_and_condition = skip_indexes.useful_indices[idx]; + const auto & stat = useful_indices_stat[idx]; const auto & index_name = index_and_condition.index->index.name; LOG_DEBUG( log, "Index {} has dropped {}/{} granules.", backQuote(index_name), - index_and_condition.stat.granules_dropped, - index_and_condition.stat.total_granules); + stat.granules_dropped, + stat.total_granules); std::string description = index_and_condition.index->index.type + " GRANULARITY " + std::to_string(index_and_condition.index->index.granularity); @@ -1151,25 +1108,27 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd .type = ReadFromMergeTree::IndexType::Skip, .name = index_name, .description = std::move(description), - .num_parts_after = index_and_condition.stat.total_parts - index_and_condition.stat.parts_dropped, - .num_granules_after = index_and_condition.stat.total_granules - index_and_condition.stat.granules_dropped}); + .num_parts_after = stat.total_parts - stat.parts_dropped, + .num_granules_after = stat.total_granules - stat.granules_dropped}); } - for (const auto & [type_with_granularity, index_and_condition] : merged_indices) + for (size_t idx = 0; idx < skip_indexes.merged_indices.size(); ++idx) { + const auto & index_and_condition = skip_indexes.merged_indices[idx]; + const auto & stat = merged_indices_stat[idx]; const auto & index_name = "Merged"; LOG_DEBUG(log, "Index {} has dropped {}/{} granules.", backQuote(index_name), - index_and_condition.stat.granules_dropped, index_and_condition.stat.total_granules); + stat.granules_dropped, stat.total_granules); - std::string description = "MERGED GRANULARITY " + std::to_string(type_with_granularity.second); + std::string description = "MERGED GRANULARITY " + std::to_string(index_and_condition.indices.at(0)->index.granularity); index_stats.emplace_back(ReadFromMergeTree::IndexStat{ .type = ReadFromMergeTree::IndexType::Skip, .name = index_name, .description = std::move(description), - .num_parts_after = index_and_condition.stat.total_parts - index_and_condition.stat.parts_dropped, - .num_granules_after = index_and_condition.stat.total_granules - index_and_condition.stat.granules_dropped}); + .num_parts_after = stat.total_parts - stat.parts_dropped, + .num_granules_after = stat.total_granules - stat.granules_dropped}); } return parts_with_ranges; @@ -1291,7 +1250,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried); - std::optional key_condition; + std::optional indexes; return ReadFromMergeTree::selectRangesToRead( std::move(parts), prewhere_info, @@ -1306,7 +1265,7 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar real_column_names, sample_factor_column_queried, log, - key_condition); + indexes); } QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts( diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index a337574bb64..8c8ce59bebe 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -189,9 +189,9 @@ public: static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes( MergeTreeData::DataPartsVector && parts, StorageMetadataPtr metadata_snapshot, - const SelectQueryInfo & query_info, const ContextPtr & context, const KeyCondition & key_condition, + const UsefulSkipIndexes & skip_indexes, const MergeTreeReaderSettings & reader_settings, Poco::Logger * log, size_t num_streams, diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 5e186c25b83..2bd9db12b93 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -310,14 +310,24 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (functionIsInOrGlobalInOperator(function_name)) { + //std::cerr << StackTrace().toString() << std::endl; + auto future_set = rhs_argument.tryGetPreparedSet(); + + //std::cerr << "==== Finding set for MergeTreeBF " << bool(future_set) << std::endl; + if (future_set && !future_set->isReady()) + { + //std::cerr << "==== not ready, building " << std::endl; future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); + } ConstSetPtr prepared_set; if (future_set) prepared_set = future_set->get(); + //std::cerr << "==== Prep set for MergeTreeBF " << bool(prepared_set) << ' ' << (prepared_set ? prepared_set->hasExplicitSetElements() : false) << std::endl; + if (prepared_set && prepared_set->hasExplicitSetElements()) { const auto prepared_info = getPreparedSetInfo(prepared_set); diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 6ffba0ad029..a64f81807ae 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -655,7 +655,12 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( if (key_tuple_mapping.empty()) return false; + //std::cerr << "==== Finding set for MergeTreeConditionInverted\n"; + auto future_set = rhs.tryGetPreparedSet(); + + //std::cerr << "==== Set for MergeTreeConditionInverted" << bool(future_set) << std::endl; + if (future_set && !future_set->isReady()) future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index e8843ff1489..cc7ec45be6a 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -298,11 +298,29 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const if (ast_node && prepared_sets) { - return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); + auto hash = ast_node->getTreeHash(); + auto key = PreparedSetKey::forSubquery(hash); + + // std::cerr << ".........Getting from AST \n" << ast_node->dumpTree() << std::endl + // << key.toString() << std::endl; + + for (const auto & [k, v] : prepared_sets->getSets()) + { + // std::cerr << "........... " << k.toString() << std::endl; + if (k.ast_hash == hash) + return v; + } + + //return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); } else if (dag_node) { + + // std::cerr << "...........Getting from DAG\n"; const auto * node_without_alias = getNodeWithoutAlias(dag_node); + // std::cerr << ".......... node_without_alias : " << node_without_alias->result_name + // << ' ' << node_without_alias->result_type->getName() + // << ' ' << (node_without_alias->column ? node_without_alias->column->getName() : "") << std::endl; return tryGetSetFromDAGNode(node_without_alias); } diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index fb895d04b8f..a8eb00adc87 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -264,5 +264,7 @@ struct SelectQueryInfo { return input_order_info ? input_order_info : (projection ? projection->input_order_info : nullptr); } + + bool isFinal() const; }; } From 9db7e8ed62415aac040ee5d220d6dabc749ccc7b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 May 2023 20:47:35 +0000 Subject: [PATCH 012/171] Fixing build. --- src/Storages/SelectQueryInfo.cpp | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 src/Storages/SelectQueryInfo.cpp diff --git a/src/Storages/SelectQueryInfo.cpp b/src/Storages/SelectQueryInfo.cpp new file mode 100644 index 00000000000..665da7fee70 --- /dev/null +++ b/src/Storages/SelectQueryInfo.cpp @@ -0,0 +1,16 @@ +#include +#include + +namespace DB +{ + +bool SelectQueryInfo::isFinal() const +{ + if (table_expression_modifiers) + return table_expression_modifiers->hasFinal(); + + const auto & select = query->as(); + return select.final(); +} + +} From d8f39b8df1ef608b9f78844a6665495be3cbec33 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 24 May 2023 17:53:37 +0000 Subject: [PATCH 013/171] Fixing more tests. --- src/Interpreters/ActionsVisitor.cpp | 8 ++++++ .../ClusterProxy/SelectStreamFactory.cpp | 8 +++--- .../ClusterProxy/SelectStreamFactory.h | 6 ++++- .../ClusterProxy/executeQuery.cpp | 1 + src/Interpreters/GlobalSubqueriesVisitor.h | 3 ++- src/Interpreters/PreparedSets.cpp | 6 ++++- .../QueryPlan/DistributedCreateLocalPlan.cpp | 3 ++- .../QueryPlan/DistributedCreateLocalPlan.h | 4 +++ src/Processors/QueryPlan/ReadFromRemote.cpp | 6 +++-- src/Processors/QueryPlan/ReadFromRemote.h | 2 ++ src/Storages/StorageDistributed.cpp | 3 ++- src/Storages/StorageMergeTree.cpp | 3 ++- src/Storages/StorageReplicatedMergeTree.cpp | 3 ++- src/Storages/VirtualColumnUtils.cpp | 27 +++++++++++++++++++ 14 files changed, 71 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index dcf6c4a461f..1405568aa71 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -3,6 +3,7 @@ #include #include #include +#include "Parsers/queryToString.h" #include #include @@ -1392,8 +1393,15 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool { if (no_subqueries) return {}; + //std::cerr << queryToString(right_in_operand) << std::endl; auto set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); + // std::cerr << set_key.toString() << std::endl; + // std::cerr << data.prepared_sets->getSets().size() << std::endl; + // std::cerr << reinterpret_cast(data.prepared_sets.get()) << std::endl; + // for (const auto & [k, v] : data.prepared_sets->getSets()) + // std::cerr << "... " << k.toString(); + if (auto set = data.prepared_sets->getFuture(set_key)) return set; diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 0cf3f360994..0f8a725e144 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -92,11 +92,13 @@ SelectStreamFactory::SelectStreamFactory( const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, const StorageSnapshotPtr & storage_snapshot_, - QueryProcessingStage::Enum processed_stage_) + QueryProcessingStage::Enum processed_stage_, + PreparedSetsPtr prepared_sets_) : header(header_), objects_by_shard(objects_by_shard_), storage_snapshot(storage_snapshot_), - processed_stage(processed_stage_) + processed_stage(processed_stage_), + prepared_sets(std::move(prepared_sets_)) { } @@ -117,7 +119,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { local_plans.emplace_back(createLocalPlan( - query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); + query_ast, header, context, processed_stage, prepared_sets, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); }; auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index f1a8b3e0984..b19012ddba6 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -26,6 +26,8 @@ using QueryPlanPtr = std::unique_ptr; struct StorageID; +class PreparedSets; +using PreparedSetsPtr = std::shared_ptr; namespace ClusterProxy { @@ -67,7 +69,8 @@ public: const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, const StorageSnapshotPtr & storage_snapshot_, - QueryProcessingStage::Enum processed_stage_); + QueryProcessingStage::Enum processed_stage_, + PreparedSetsPtr prepared_sets_); void createForShard( const Cluster::ShardInfo & shard_info, @@ -92,6 +95,7 @@ public: const ColumnsDescriptionByShardNum objects_by_shard; const StorageSnapshotPtr storage_snapshot; QueryProcessingStage::Enum processed_stage; + PreparedSetsPtr prepared_sets; }; } diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index e2f1dfe8ba7..a9cf3d55392 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -226,6 +226,7 @@ void executeQuery( std::move(remote_shards), header, processed_stage, + stream_factory.prepared_sets, main_table, table_func_ptr, new_context, diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 3a846bb4bc3..a872af529aa 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -181,7 +181,7 @@ public: // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); - auto set_key = PreparedSetKey::forSubquery(subquery_or_table_name->getTreeHash()); + auto set_key = PreparedSetKey::forSubquery(database_and_table_name->getTreeHash()); if (!prepared_sets->getFuture(set_key)) { @@ -190,6 +190,7 @@ public: subquery_for_set.table = std::move(external_storage); subquery_for_set.createSource(*interpreter); + //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); } else diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index cd6b2a81ba0..3b63d942404 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -140,6 +140,10 @@ FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryF if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + // std::cerr << key.toString() << std::endl; + // std::cerr << "========= PreparedSets::addFromSubquery\n"; + // std::cerr << StackTrace().toString() << std::endl; + subqueries.emplace(id, std::move(from_subquery)); return it->second; } @@ -226,7 +230,7 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (set) return nullptr; - //std::cerr << StackTrace().toString() << std::endl; + // std::cerr << StackTrace().toString() << std::endl; auto set_cache = context->getPreparedSetsCache(); if (set_cache) diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 9b9cc221ca8..62e369659d1 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -43,6 +43,7 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, + PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, @@ -98,7 +99,7 @@ std::unique_ptr createLocalPlan( } else { - auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); + auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options, prepared_sets); interpreter.buildQueryPlan(*query_plan); } diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h index 1afdc07fa4d..cf59027a33f 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h @@ -10,11 +10,15 @@ namespace DB { +class PreparedSets; +using PreparedSetsPtr = std::shared_ptr; + std::unique_ptr createLocalPlan( const ASTPtr & query_ast, const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, + PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index ed740e3e242..16cb06a94d6 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -97,6 +97,7 @@ ReadFromRemote::ReadFromRemote( ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, + PreparedSetsPtr prepared_sets_, StorageID main_table_, ASTPtr table_func_ptr_, ContextMutablePtr context_, @@ -109,6 +110,7 @@ ReadFromRemote::ReadFromRemote( : ISourceStep(DataStream{.header = std::move(header_)}) , shards(std::move(shards_)) , stage(stage_) + , prepared_sets(std::move(prepared_sets_)) , main_table(std::move(main_table_)) , table_func_ptr(std::move(table_func_ptr_)) , context(std::move(context_)) @@ -150,7 +152,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream my_context = context, my_throttler = throttler, my_main_table = main_table, my_table_func_ptr = table_func_ptr, my_scalars = scalars, my_external_tables = external_tables, - my_stage = stage, local_delay = shard.local_delay, + my_stage = stage, my_prepared_sets = prepared_sets, local_delay = shard.local_delay, add_agg_info, add_totals, add_extremes, async_read, async_query_sending]() mutable -> QueryPipelineBuilder { @@ -185,7 +187,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream if (try_results.empty() || local_delay < max_remote_delay) { auto plan = createLocalPlan( - query, header, my_context, my_stage, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); + query, header, my_context, my_stage, my_prepared_sets, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); return std::move(*plan->buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(my_context), diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index d4005d81f1b..405b5727ff2 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -26,6 +26,7 @@ public: ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, + PreparedSetsPtr prepared_sets_, StorageID main_table_, ASTPtr table_func_ptr_, ContextMutablePtr context_, @@ -46,6 +47,7 @@ public: private: ClusterProxy::SelectStreamFactory::Shards shards; QueryProcessingStage::Enum stage; + PreparedSetsPtr prepared_sets; StorageID main_table; ASTPtr table_func_ptr; ContextMutablePtr context; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 86a2599b49c..f45e247f8e7 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1098,7 +1098,8 @@ void StorageDistributed::read( header, snapshot_data.objects_by_shard, storage_snapshot, - processed_stage); + processed_stage, + query_info.prepared_sets); auto settings = local_context->getSettingsRef(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2c19d3ba122..4600532231f 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -231,7 +231,8 @@ void StorageMergeTree::read( header, {}, storage_snapshot, - processed_stage); + processed_stage, + query_info.prepared_sets); ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index d9c8f09ccf1..ab78ea2f0a4 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4663,7 +4663,8 @@ void StorageReplicatedMergeTree::read( header, {}, storage_snapshot, - processed_stage); + processed_stage, + query_info.prepared_sets); ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 31fa1cc6a7e..be2206a78e9 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -23,6 +23,8 @@ #include #include #include +#include +#include #include @@ -202,6 +204,31 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex //buildSets(expression_ast, analyzer); ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes); + for (const auto & node : actions->getNodes()) + { + if (node.type == ActionsDAG::ActionType::COLUMN) + { + const ColumnSet * column_set = checkAndGetColumnConstData(node.column.get()); + if (!column_set) + column_set = checkAndGetColumn(node.column.get()); + + if (column_set) + { + auto future_set = column_set->getData(); + if (!future_set->isFilled()) + { + auto plan = future_set->build(context); + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + } + } + } + } + Block block_with_filter = block; actions->execute(block_with_filter); From 30ff5113d98141ab4adb05ad08b7ffb48888c33d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 May 2023 13:33:52 +0000 Subject: [PATCH 014/171] Fixing more tests. --- src/Functions/in.cpp | 8 +- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/PreparedSets.cpp | 70 +++++++++++------ src/Interpreters/PreparedSets.h | 17 +++-- src/Interpreters/Set.cpp | 76 +++++++++++-------- src/Interpreters/Set.h | 18 ++++- src/Planner/CollectSets.cpp | 2 +- .../CreateSetAndFilterOnTheFlyTransform.cpp | 2 +- src/Storages/KVStorageUtils.cpp | 4 +- src/Storages/MergeTree/KeyCondition.cpp | 8 +- .../MergeTreeIndexConditionBloomFilter.cpp | 2 +- .../MergeTree/MergeTreeIndexFullText.cpp | 2 +- .../MergeTree/MergeTreeIndexInverted.cpp | 2 +- .../System/StorageSystemZooKeeper.cpp | 4 +- 14 files changed, 136 insertions(+), 81 deletions(-) diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index eb623951bf3..6a88a413c63 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -127,12 +127,12 @@ public: } auto future_set = column_set->getData(); - if (!future_set || !future_set->isFilled()) + if (!future_set || !future_set->isReady()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName()); - if (auto * for_tuple = typeid_cast(future_set.get())) - if (!for_tuple->isReady()) - for_tuple->buildForTuple(size_limits, transform_null_in); + // if (auto * for_tuple = typeid_cast(future_set.get())) + // if (!for_tuple->isReady()) + // for_tuple->buildForTuple(size_limits, transform_null_in); auto set = future_set->get(); auto set_types = set->getDataTypes(); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 1405568aa71..da8666466fc 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -458,7 +458,7 @@ FutureSetPtr makeExplicitSet( else block = createBlockForSet(left_arg_type, right_arg, set_element_types, context); - return prepared_sets.addFromTuple(set_key, block); + return prepared_sets.addFromTuple(set_key, block, context->getSettings()); } class ScopeStack::Index diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 3b63d942404..eebc59e85d5 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int SET_SIZE_LIMIT_EXCEEDED; } PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) @@ -120,9 +121,9 @@ FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set return it->second; } -FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block) +FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings) { - auto from_tuple = std::make_shared(std::move(block)); + auto from_tuple = std::make_shared(std::move(block), settings); auto [it, inserted] = sets.emplace(key, std::move(from_tuple)); if (!inserted) @@ -292,7 +293,25 @@ SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordere return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); } -FutureSetFromTuple::FutureSetFromTuple(Block block_) : block(std::move(block_)) {} +FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) +{ + bool create_ordered_set = false; + auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); + + Columns columns; + columns.reserve(block.columns()); + for (const auto & column : block) + columns.emplace_back(column.column); + + set_key_columns.filter = ColumnUInt8::create(block.rows()); + + set->initSetElements(); + set->insertFromColumns(columns, set_key_columns); + set->finishInsert(); + //block(std::move(block_)) +} FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} @@ -301,35 +320,40 @@ FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) { SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, true); - fill(size_limits, settings.transform_null_in, true); + auto limits = getSizeLimitsForSet(settings, true); + + if (!limits.check(set->getTotalRowCount(), set->getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED)) + return nullptr; + + set->initSetElements(); + set->appendSetElements(set_key_columns); return set; } -std::unique_ptr FutureSetFromTuple::build(const ContextPtr & context) +std::unique_ptr FutureSetFromTuple::build(const ContextPtr &) { - const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, false); - fill(size_limits, settings.transform_null_in, false); + // const auto & settings = context->getSettingsRef(); + // auto size_limits = getSizeLimitsForSet(settings, false); + // fill(size_limits, settings.transform_null_in, false); return nullptr; } -void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) -{ - fill(size_limits, transform_null_in, false); -} +// void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) +// { +// fill(size_limits, transform_null_in, false); +// } -void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) -{ - //std::cerr << StackTrace().toString() << std::endl; +// void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) +// { +// //std::cerr << StackTrace().toString() << std::endl; - if (set) - return; +// if (set) +// return; - set = std::make_shared(size_limits, create_ordered_set, transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - set->insertFromBlock(block.getColumnsWithTypeAndName()); - set->finishInsert(); -} +// set = std::make_shared(size_limits, create_ordered_set, transform_null_in); +// set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); +// set->insertFromBlock(block.getColumnsWithTypeAndName()); +// set->finishInsert(); +// } }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index ef7aba38f24..35bbc1e12fb 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -80,24 +80,23 @@ using FutureSetPtr = std::shared_ptr; class FutureSetFromTuple final : public FutureSet { public: - FutureSetFromTuple(Block block_); + FutureSetFromTuple(Block block, const Settings & settings); - bool isReady() const override { return set != nullptr; } + bool isReady() const override { return true; } bool isFilled() const override { return true; } SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - std::unique_ptr build(const ContextPtr & context) override; + std::unique_ptr build(const ContextPtr &) override; - void buildForTuple(SizeLimits size_limits, bool transform_null_in); +/// void buildForTuple(SizeLimits size_limits, bool transform_null_in); private: - Block block; - SetPtr set; + Set::SetKeyColumns set_key_columns; - void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); + //void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); }; /// Information on how to build set for the [GLOBAL] IN section. @@ -145,6 +144,8 @@ public: return nullptr; auto plan = buildPlan(context, true); + if (!plan) + return nullptr; auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); @@ -249,7 +250,7 @@ public: // SizeLimits set_size_limit, bool transform_null_in); FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); - FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block); + FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery); void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index a7bea63bd99..5adf3d07353 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -156,25 +156,46 @@ void Set::setHeader(const ColumnsWithTypeAndName & header) { /// Create empty columns with set values in advance. /// It is needed because set may be empty, so method 'insertFromBlock' will be never called. - set_elements.reserve(keys_size); - for (const auto & type : set_elements_types) - set_elements.emplace_back(type->createColumn()); + initSetElements(); } /// Choose data structure to use for the set. data.init(data.chooseMethod(key_columns, key_sizes)); } +void Set::initSetElements() +{ + set_elements.reserve(keys_size); + for (const auto & type : set_elements_types) + set_elements.emplace_back(type->createColumn()); +} + bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns) { Columns cols; cols.reserve(columns.size()); for (const auto & column : columns) cols.emplace_back(column.column); - return insertFromBlock(cols); + return insertFromColumns(cols); } -bool Set::insertFromBlock(const Columns & columns) +bool Set::insertFromColumns(const Columns & columns) +{ + size_t rows = columns.at(0)->size(); + + SetKeyColumns holder; + /// Filter to extract distinct values from the block. + if (fill_set_elements) + holder.filter = ColumnUInt8::create(rows); + + bool inserted = insertFromColumns(columns, holder); + if (inserted && fill_set_elements) + appendSetElements(holder); + + return inserted; +} + +bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) { std::lock_guard lock(rwlock); @@ -183,15 +204,13 @@ bool Set::insertFromBlock(const Columns & columns) ColumnRawPtrs key_columns; key_columns.reserve(keys_size); - - /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - Columns materialized_columns; + holder.key_columns.reserve(keys_size); /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); - key_columns.emplace_back(materialized_columns.back().get()); + holder.key_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); + key_columns.emplace_back(holder.key_columns.back().get()); } size_t rows = columns.at(0)->size(); @@ -202,40 +221,37 @@ bool Set::insertFromBlock(const Columns & columns) if (!transform_null_in) null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); - /// Filter to extract distinct values from the block. - ColumnUInt8::MutablePtr filter; - if (fill_set_elements) - filter = ColumnUInt8::create(rows); - switch (data.type) { case SetVariants::Type::EMPTY: break; #define M(NAME) \ case SetVariants::Type::NAME: \ - insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, filter ? &filter->getData() : nullptr); \ + insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ break; APPLY_FOR_SET_VARIANTS(M) #undef M } - if (fill_set_elements) - { - for (size_t i = 0; i < keys_size; ++i) - { - auto filtered_column = key_columns[i]->filter(filter->getData(), rows); - if (set_elements[i]->empty()) - set_elements[i] = filtered_column; - else - set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size()); - if (transform_null_in && null_map_holder) - set_elements[i]->insert(Null{}); - } - } - return limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); } +void Set::appendSetElements(SetKeyColumns & holder) +{ + //std::cerr << "========= " << keys_size << ' ' << holder.key_columns.size() << std::endl; + size_t rows = holder.key_columns.at(0)->size(); + for (size_t i = 0; i < keys_size; ++i) + { + auto filtered_column = holder.key_columns[i]->filter(holder.filter->getData(), rows); + if (set_elements[i]->empty()) + set_elements[i] = filtered_column; + else + set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size()); + if (transform_null_in && holder.null_map_holder) + set_elements[i]->insert(Null{}); + } +} + void Set::checkIsCreated() const { if (!is_created.load()) diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index fff5fa4e1b1..cb47fde7f7d 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -45,9 +45,23 @@ public: void setHeader(const ColumnsWithTypeAndName & header); /// Returns false, if some limit was exceeded and no need to insert more data. - bool insertFromBlock(const Columns & columns); + bool insertFromColumns(const Columns & columns); bool insertFromBlock(const ColumnsWithTypeAndName & columns); + + struct SetKeyColumns + { + //ColumnRawPtrs key_columns; + /// The constant columns to the right of IN are not supported directly. For this, they first materialize. + Columns key_columns; + ColumnPtr null_map_holder; + ColumnUInt8::MutablePtr filter; + }; + + void initSetElements(); + bool insertFromColumns(const Columns & columns, SetKeyColumns & holder); + void appendSetElements(SetKeyColumns & holder); + /// Call after all blocks were inserted. To get the information that set is already created. void finishInsert() { is_created = true; } @@ -68,7 +82,7 @@ public: const DataTypes & getDataTypes() const { return data_types; } const DataTypes & getElementsTypes() const { return set_elements_types; } - bool hasExplicitSetElements() const { return fill_set_elements; } + bool hasExplicitSetElements() const { return fill_set_elements || (!set_elements.empty() && set_elements.front()->size() == data.getTotalRowCount()); } Columns getSetElements() const { checkIsCreated(); return { set_elements.begin(), set_elements.end() }; } void checkColumnsNumber(size_t num_key_columns) const; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 6c970e0e91b..d66424f89ec 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -82,7 +82,7 @@ public: auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); - sets.addFromTuple(set_key, std::move(set)); + sets.addFromTuple(set_key, std::move(set), settings); //planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); } diff --git a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp index 4278eb8e8b2..59c4b9a6a87 100644 --- a/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp +++ b/src/Processors/Transforms/CreateSetAndFilterOnTheFlyTransform.cpp @@ -106,7 +106,7 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk) if (chunk.getNumRows()) { Columns key_columns = getColumnsByIndices(chunk, key_column_indices); - bool limit_exceeded = !set->insertFromBlock(key_columns); + bool limit_exceeded = !set->insertFromColumns(key_columns); if (limit_exceeded) { auto prev_state = set->state.exchange(SetWithState::State::Suspended); diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 281236e631e..f4a4b6e9af1 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -76,8 +76,8 @@ bool traverseASTFilter( if (!future_set) return false; - if (!future_set->isReady()) - future_set->buildOrderedSetInplace(context); + //if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); auto set = future_set->get(); if (!set) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 4b053a37aa2..7fcc111ced9 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1212,11 +1212,11 @@ bool KeyCondition::tryPrepareSetIndex( // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); - if (!future_set->isReady()) - { + //if (!future_set->isReady()) + //{ // LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); - future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); - } + future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); + //} auto prepared_set = future_set->get(); if (!prepared_set) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 2bd9db12b93..3d0883a1241 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -316,7 +316,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo //std::cerr << "==== Finding set for MergeTreeBF " << bool(future_set) << std::endl; - if (future_set && !future_set->isReady()) + if (future_set) // && !future_set->isReady()) { //std::cerr << "==== not ready, building " << std::endl; future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 5e1d23df3c7..d14ff4fa20b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -625,7 +625,7 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( return false; auto future_set = right_argument.tryGetPreparedSet(data_types); - if (future_set && !future_set->isReady()) + if (future_set) // && !future_set->isReady()) future_set->buildOrderedSetInplace(right_argument.getTreeContext().getQueryContext()); ConstSetPtr prepared_set; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index a64f81807ae..1d3b923056a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -661,7 +661,7 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( //std::cerr << "==== Set for MergeTreeConditionInverted" << bool(future_set) << std::endl; - if (future_set && !future_set->isReady()) + if (future_set) // && !future_set->isReady()) future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); ConstSetPtr prepared_set; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 0f45f6825f6..34b463eadee 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -317,8 +317,8 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP if (!future_set) return; - if (!future_set->isReady()) - future_set->buildOrderedSetInplace(context); + //if (!future_set->isReady()) + future_set->buildOrderedSetInplace(context); auto set = future_set->get(); if (!set) From 7ebe19f5fb1390bc5f2dac3a5cca7db9c76bdd98 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 May 2023 19:18:11 +0000 Subject: [PATCH 015/171] Fixing more tests --- src/Interpreters/ActionsVisitor.cpp | 7 ++- src/Interpreters/Context.cpp | 15 +++++ src/Interpreters/Context.h | 1 + src/Interpreters/DatabaseCatalog.h | 3 + src/Interpreters/GlobalSubqueriesVisitor.h | 19 ++++--- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/PreparedSets.cpp | 55 +++++++++++++++---- src/Interpreters/PreparedSets.h | 35 ++++-------- src/Planner/CollectSets.cpp | 3 +- src/Planner/Planner.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 8 ++- src/Processors/QueryPlan/CreatingSetsStep.cpp | 4 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 4 +- .../QueryPlan/ReadFromMemoryStorageStep.cpp | 6 +- .../QueryPlan/ReadFromMemoryStorageStep.h | 2 + .../Transforms/CreatingSetsTransform.cpp | 13 +++-- .../Transforms/CreatingSetsTransform.h | 5 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 4 +- src/QueryPipeline/QueryPipelineBuilder.h | 5 +- src/Storages/StorageMemory.cpp | 2 +- src/Storages/StorageMemory.h | 2 + 21 files changed, 132 insertions(+), 65 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index da8666466fc..133fcbbfe87 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1405,6 +1405,8 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (auto set = data.prepared_sets->getFuture(set_key)) return set; + FutureSetPtr external_table_set; + /// A special case is if the name of the table is specified on the right side of the IN statement, /// and the table has the type Set (a previously prepared set). if (identifier) @@ -1417,6 +1419,9 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (StorageSet * storage_set = dynamic_cast(table.get())) return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } + + if (auto tmp_table = data.getContext()->findExternalTable(table_id.getShortName())) + external_table_set = tmp_table->future_set; } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. @@ -1438,7 +1443,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool subquery_for_set.createSource(*interpreter); } - return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); + return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), std::move(external_table_set)); } else { diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index d9f450191bc..f8fe99fbcc6 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1319,6 +1319,21 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder & external_tables_mapping.emplace(table_name, std::make_shared(std::move(temporary_table))); } +std::shared_ptr Context::findExternalTable(const String & table_name) const +{ + if (isGlobalContext()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have external tables"); + + std::shared_ptr holder; + { + auto lock = getLock(); + auto iter = external_tables_mapping.find(table_name); + if (iter == external_tables_mapping.end()) + return {}; + holder = iter->second; + } + return holder; +} std::shared_ptr Context::removeExternalTable(const String & table_name) { diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 1be662e0958..1a0ee891699 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -611,6 +611,7 @@ public: Tables getExternalTables() const; void addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table); + std::shared_ptr findExternalTable(const String & table_name) const; std::shared_ptr removeExternalTable(const String & table_name); const Scalars & getScalars() const; diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index 51e9fbdb936..258ea2dee7c 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -79,6 +79,8 @@ private: using DDLGuardPtr = std::unique_ptr; +class FutureSet; +using FutureSetPtr = std::shared_ptr; /// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID. /// Such table can be accessed from everywhere by its ID. @@ -111,6 +113,7 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext IDatabase * temporary_tables = nullptr; UUID id = UUIDHelpers::Nil; + FutureSetPtr future_set; }; ///TODO maybe remove shared_ptr from here? diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index a872af529aa..195839da04f 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -64,6 +64,7 @@ public: void addExternalStorage(ASTPtr & ast, bool set_alias = false) { + // std::cerr << "=============== addExternalStorage is remote " << is_remote << std::endl; /// With nondistributed queries, creating temporary tables does not make sense. if (!is_remote) return; @@ -163,10 +164,10 @@ public: /// We need to materialize external tables immediately because reading from distributed /// tables might generate local plans which can refer to external tables during index /// analysis. It's too late to populate the external table via CreatingSetsTransform. - if (is_explain) - { - /// Do not materialize external tables if it's explain statement. - } + // if (is_explain) + // { + // /// Do not materialize external tables if it's explain statement. + // } // else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) // { // auto external_table = external_storage_holder->getTable(); @@ -176,13 +177,15 @@ public: // CompletedPipelineExecutor executor(io.pipeline); // executor.execute(); // } - else + // else { // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); auto set_key = PreparedSetKey::forSubquery(database_and_table_name->getTreeHash()); + // std::cerr << "====== Adding key " << set_key.toString() << std::endl; + if (!prepared_sets->getFuture(set_key)) { SubqueryForSet subquery_for_set; @@ -191,10 +194,12 @@ public: subquery_for_set.createSource(*interpreter); //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; - prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set)); + auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + external_storage_holder->future_set = std::move(future_set); } else - prepared_sets->addStorageToSubquery(key, std::move(external_storage)); + throw Exception(ErrorCodes::LOGICAL_ERROR, "!!!!!!!!"); + //prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8d305c07ce9..28a0e27384a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3088,7 +3088,7 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - auto subqueries = prepared_sets->detachSubqueries(context); + auto subqueries = prepared_sets->detachSubqueries(); if (!subqueries.empty()) { diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index eebc59e85d5..f2b70462227 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -101,14 +101,14 @@ String PreparedSetKey::toString() const /// If the subquery is not associated with any set, create default-constructed SubqueryForSet. /// It's aimed to fill external table passed to SubqueryForSet::createSource. -void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) -{ - auto it = subqueries.find(subquery_id); - if (it == subqueries.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); +// void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) +// { +// auto it = subqueries.find(subquery_id); +// if (it == subqueries.end()) +// throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); - it->second->addStorage(std::move(storage)); -} +// it->second->addStorage(std::move(storage)); +// } FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set_) { @@ -132,10 +132,10 @@ FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, return it->second; } -FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery) +FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set) { auto id = subquery.key; - auto from_subquery = std::make_shared(std::move(subquery)); + auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set)); auto [it, inserted] = sets.emplace(key, from_subquery); if (!inserted) @@ -145,7 +145,7 @@ FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryF // std::cerr << "========= PreparedSets::addFromSubquery\n"; // std::cerr << StackTrace().toString() << std::endl; - subqueries.emplace(id, std::move(from_subquery)); + subqueries.emplace_back(SetAndName{.name = id, .set = std::move(from_subquery)}); return it->second; } @@ -176,7 +176,7 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return res; // } -PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries(const ContextPtr &) +PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() { auto res = std::move(subqueries); subqueries = SubqueriesForSets(); @@ -226,6 +226,36 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo return promise_to_fill_set; } +SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) +{ + if (!context->getSettingsRef().use_index_for_in_with_subqueries) + return nullptr; + + if (set) + { + if (set->hasExplicitSetElements()) + return set; + + return nullptr; + } + + if (external_table_set) + return set = external_table_set->buildOrderedSetInplace(context); + + auto plan = buildPlan(context, true); + if (!plan) + return nullptr; + + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + + return set; +} + std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) { if (set) @@ -313,7 +343,8 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) //block(std::move(block_)) } -FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_) : subquery(std::move(subquery_)) {} +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_) + : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)) {} FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 35bbc1e12fb..c510e690e25 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -132,30 +132,13 @@ public: class FutureSetFromSubquery : public FutureSet { public: - FutureSetFromSubquery(SubqueryForSet subquery_); + FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_); bool isReady() const override { return set != nullptr; } bool isFilled() const override { return isReady(); } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr & context) override - { - if (!context->getSettingsRef().use_index_for_in_with_subqueries) - return nullptr; - - auto plan = buildPlan(context, true); - if (!plan) - return nullptr; - - auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); - - CompletedPipelineExecutor executor(pipeline); - executor.execute(); - - return set; - } + SetPtr buildOrderedSetInplace(const ContextPtr & context) override; std::unique_ptr build(const ContextPtr & context) override { @@ -167,6 +150,7 @@ public: private: SetPtr set; SubqueryForSet subquery; + FutureSetPtr external_table_set; std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; @@ -244,16 +228,21 @@ struct PreparedSetKey class PreparedSets { public: - using SubqueriesForSets = std::unordered_map>; + struct SetAndName + { + String name; + std::shared_ptr set; + }; + using SubqueriesForSets = std::vector; // SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, // SizeLimits set_size_limit, bool transform_null_in); FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery); + FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set); - void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); + //void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); FutureSetPtr getFuture(const PreparedSetKey & key) const; //SubqueryForSet & getSubquery(const String & subquery_id); @@ -262,7 +251,7 @@ public: /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. /// SetPtr would still be available for consumers of PreparedSets. - SubqueriesForSets detachSubqueries(const ContextPtr &); + SubqueriesForSets detachSubqueries(); /// Returns all sets that match the given ast hash not checking types /// Used in KeyCondition and MergeTreeIndexConditionBloomFilter to make non exact match for types in PreparedSetKey diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index d66424f89ec..19dc569ff0c 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -107,7 +107,8 @@ public: subquery_for_set.key = planner_context.createSetKey(in_second_argument); subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); - sets.addFromSubquery(set_key, std::move(subquery_for_set)); + /// TODO + sets.addFromSubquery(set_key, std::move(subquery_for_set), nullptr); //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index b1780212e51..5c3fc82a80b 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -1468,7 +1468,7 @@ void Planner::buildPlanForQueryNode() if (!select_query_options.only_analyze) { - auto subqueries = planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()); + auto subqueries = planner_context->getPreparedSets().detachSubqueries(); if (!subqueries.empty()) { diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index e0844a6d2b1..bbac9bf5c4b 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -632,7 +632,13 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma DataTypes set_element_types; auto in_second_argument_node_type = in_second_argument->getNodeType(); - if (!(in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION)) + // std::cerr << "=========== " << in_second_argument->getNodeTypeName() << std::endl; + bool subquery_or_table = + in_second_argument_node_type == QueryTreeNodeType::QUERY || + in_second_argument_node_type == QueryTreeNodeType::UNION || + in_second_argument_node_type == QueryTreeNodeType::TABLE; + + if (!subquery_or_table) { set_element_types = {in_first_argument->getResultType()}; const auto * left_tuple_type = typeid_cast(set_element_types.front().get()); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 34018b2144c..38506412531 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -48,7 +48,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context); + pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context->getPreparedSetsCache()); } void CreatingSetStep::updateOutputStream() @@ -189,7 +189,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, if (!prepared_sets || prepared_sets->empty()) return; - addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(context), context); + addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(), context); } DelayedCreatingSetsStep::DelayedCreatingSetsStep( diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 62e369659d1..8e2221d564d 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -43,7 +43,7 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, - PreparedSetsPtr prepared_sets, + [[maybe_unused]] PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, @@ -99,7 +99,7 @@ std::unique_ptr createLocalPlan( } else { - auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options, prepared_sets); + auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); //, prepared_sets); interpreter.buildQueryPlan(*query_plan); } diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp index 2080c31d253..4ad3cc7373a 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp @@ -96,11 +96,13 @@ private: }; ReadFromMemoryStorageStep::ReadFromMemoryStorageStep(const Names & columns_to_read_, + StoragePtr storage_, const StorageSnapshotPtr & storage_snapshot_, const size_t num_streams_, const bool delay_read_for_global_sub_queries_) : SourceStepWithFilter(DataStream{.header=storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}), columns_to_read(columns_to_read_), + storage(std::move(storage_)), storage_snapshot(storage_snapshot_), num_streams(num_streams_), delay_read_for_global_sub_queries(delay_read_for_global_sub_queries_) @@ -142,9 +144,9 @@ Pipe ReadFromMemoryStorageStep::makePipe() storage_snapshot, nullptr /* data */, nullptr /* parallel execution index */, - [current_data](std::shared_ptr & data_to_initialize) + [storage = storage](std::shared_ptr & data_to_initialize) { - data_to_initialize = current_data; + data_to_initialize = static_cast(*storage).data.get(); })); } diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h index cec523ed58b..1122bfbb2a5 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.h @@ -16,6 +16,7 @@ class ReadFromMemoryStorageStep final : public SourceStepWithFilter { public: ReadFromMemoryStorageStep(const Names & columns_to_read_, + StoragePtr storage_, const StorageSnapshotPtr & storage_snapshot_, size_t num_streams_, bool delay_read_for_global_sub_queries_); @@ -35,6 +36,7 @@ private: static constexpr auto name = "ReadFromMemoryStorage"; Names columns_to_read; + StoragePtr storage; StorageSnapshotPtr storage_snapshot; size_t num_streams; bool delay_read_for_global_sub_queries; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 6626d4b9795..24e17ff210f 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -27,11 +27,11 @@ CreatingSetsTransform::CreatingSetsTransform( Block out_header_, SubqueryForSet subquery_for_set_, SizeLimits network_transfer_limits_, - ContextPtr context_) + PreparedSetsCachePtr prepared_sets_cache_) : IAccumulatingTransform(std::move(in_header_), std::move(out_header_)) - , WithContext(context_) , subquery(std::move(subquery_for_set_)) , network_transfer_limits(std::move(network_transfer_limits_)) + , prepared_sets_cache(std::move(prepared_sets_cache_)) { } @@ -52,14 +52,13 @@ void CreatingSetsTransform::work() void CreatingSetsTransform::startSubquery() { /// Lookup the set in the cache if we don't need to build table. - auto ctx = context.lock(); - if (ctx && ctx->getPreparedSetsCache() && !subquery.table) + if (prepared_sets_cache && !subquery.table) { /// Try to find the set in the cache and wait for it to be built. /// Retry if the set from cache fails to be built. while (true) { - auto from_cache = ctx->getPreparedSetsCache()->findOrPromiseToBuild(subquery.key); + auto from_cache = prepared_sets_cache->findOrPromiseToBuild(subquery.key); if (from_cache.index() == 0) { promise_to_build = std::move(std::get<0>(from_cache)); @@ -89,9 +88,11 @@ void CreatingSetsTransform::startSubquery() if (subquery.table) LOG_TRACE(log, "Filling temporary table."); + // std::cerr << StackTrace().toString() << std::endl; + if (subquery.table) /// TODO: make via port - table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext())); + table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), nullptr)); done_with_set = !subquery.set; done_with_table = !subquery.table; diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index 26bbc45933d..ef586b0655c 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -23,7 +23,7 @@ class PushingPipelineExecutor; /// Don't return any data. Sets are created when Finish status is returned. /// In general, several work() methods need to be called to finish. /// Independent processors is created for each subquery. -class CreatingSetsTransform : public IAccumulatingTransform, WithContext +class CreatingSetsTransform : public IAccumulatingTransform { public: CreatingSetsTransform( @@ -31,7 +31,7 @@ public: Block out_header_, SubqueryForSet subquery_for_set_, SizeLimits network_transfer_limits_, - ContextPtr context_); + PreparedSetsCachePtr prepared_sets_cache_); ~CreatingSetsTransform() override; @@ -55,6 +55,7 @@ private: bool done_with_table = true; SizeLimits network_transfer_limits; + PreparedSetsCachePtr prepared_sets_cache; size_t rows_to_transfer = 0; size_t bytes_to_transfer = 0; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 764997e7b7e..4898501b980 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -569,7 +569,7 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe return left; } -void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context) +void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache) { resize(1); @@ -578,7 +578,7 @@ void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, Su res_header, std::move(subquery_for_set), limits, - context); + std::move(prepared_sets_cache)); InputPort * totals_port = nullptr; diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 3a5d65d4388..85b6f5c6772 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -33,6 +33,9 @@ class TableJoin; class QueryPipelineBuilder; using QueryPipelineBuilderPtr = std::unique_ptr; +class PreparedSetsCache; +using PreparedSetsCachePtr = std::shared_ptr; + class QueryPipelineBuilder { public: @@ -138,7 +141,7 @@ public: /// This is used for CreatingSets. void addPipelineBefore(QueryPipelineBuilder pipeline); - void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context); + void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache); PipelineExecutorPtr execute(); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 31e45db55cb..1392ed5c684 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -155,7 +155,7 @@ void StorageMemory::read( size_t /*max_block_size*/, size_t num_streams) { - query_plan.addStep(std::make_unique(column_names, storage_snapshot, num_streams, delay_read_for_global_subqueries)); + query_plan.addStep(std::make_unique(column_names, shared_from_this(), storage_snapshot, num_streams, delay_read_for_global_subqueries)); } diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index ce8a59b8bcd..31c8ee31f8b 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -132,6 +132,8 @@ private: std::atomic total_size_rows = 0; bool compress; + + friend class ReadFromMemoryStorageStep; }; } From 6ed7a3b73feadcf5c3d72ed4a137fccd55e10fbb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 May 2023 19:25:33 +0000 Subject: [PATCH 016/171] Fixing more tests. --- src/Interpreters/ActionsVisitor.cpp | 6 +++- src/Interpreters/DatabaseCatalog.cpp | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 4 +++ src/Interpreters/PreparedSets.cpp | 7 +++- src/Interpreters/Set.cpp | 17 ++++++---- src/Interpreters/Set.h | 3 +- .../QueryPlan/ReadFromMergeTree.cpp | 27 ++++++++++++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 3 ++ src/Storages/MergeTree/KeyCondition.cpp | 18 +++++++++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 20 ++++------- .../MergeTree/MergeTreeDataSelectExecutor.h | 3 +- .../MergeTreeIndexConditionBloomFilter.cpp | 11 ++++++ src/Storages/MergeTree/PartitionPruner.cpp | 16 +++++++++ src/Storages/MergeTree/PartitionPruner.h | 1 + .../System/StorageSystemZooKeeper.cpp | 34 +++++++++++-------- 15 files changed, 127 insertions(+), 45 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 133fcbbfe87..23c6867a868 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1420,8 +1420,12 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } - if (auto tmp_table = data.getContext()->findExternalTable(table_id.getShortName())) + // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; + if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) + { external_table_set = tmp_table->future_set; + // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; + } } /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 8d3fa91a7fe..dccf2978f49 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -110,7 +110,7 @@ TemporaryTableHolder::TemporaryTableHolder( } TemporaryTableHolder::TemporaryTableHolder(TemporaryTableHolder && rhs) noexcept - : WithContext(rhs.context), temporary_tables(rhs.temporary_tables), id(rhs.id) + : WithContext(rhs.context), temporary_tables(rhs.temporary_tables), id(rhs.id), future_set(std::move(rhs.future_set)) { rhs.id = UUIDHelpers::Nil; } diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 195839da04f..8d2f2204e84 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -30,6 +30,7 @@ namespace DB namespace ErrorCodes { extern const int WRONG_GLOBAL_SUBQUERY; + extern const int LOGICAL_ERROR; } class GlobalSubqueriesMatcher @@ -159,6 +160,8 @@ public: /*create_for_global_subquery*/ true); StoragePtr external_storage = external_storage_holder->getTable(); + // std::cerr << "......... adding external table " << external_table_name << std::endl; + external_tables.emplace(external_table_name, external_storage_holder); /// We need to materialize external tables immediately because reading from distributed @@ -195,6 +198,7 @@ public: //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + // std::cerr << "... Future set " << reinterpret_cast(external_storage_holder.get()) << " " << reinterpret_cast(future_set.get()) << std::endl; external_storage_holder->future_set = std::move(future_set); } else diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index f2b70462227..c71273e0baa 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -239,6 +239,8 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) return nullptr; } + // std::cerr << "... external_table_set " << reinterpret_cast(external_table_set.get()) << std::endl; + if (external_table_set) return set = external_table_set->buildOrderedSetInplace(context); @@ -337,7 +339,7 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set_key_columns.filter = ColumnUInt8::create(block.rows()); - set->initSetElements(); + //set->initSetElements(); set->insertFromColumns(columns, set_key_columns); set->finishInsert(); //block(std::move(block_)) @@ -350,6 +352,9 @@ FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) { SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { + if (set->hasExplicitSetElements()) + return set; + const auto & settings = context->getSettingsRef(); auto limits = getSizeLimitsForSet(settings, true); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 5adf3d07353..bd9cafc66eb 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -202,15 +202,14 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) if (data.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Method Set::setHeader must be called before Set::insertFromBlock"); - ColumnRawPtrs key_columns; - key_columns.reserve(keys_size); holder.key_columns.reserve(keys_size); + holder.materialized_columns.reserve(keys_size); /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - holder.key_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); - key_columns.emplace_back(holder.key_columns.back().get()); + holder.materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded()); + holder.key_columns.emplace_back(holder.materialized_columns.back().get()); } size_t rows = columns.at(0)->size(); @@ -219,7 +218,7 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) ConstNullMapPtr null_map{}; ColumnPtr null_map_holder; if (!transform_null_in) - null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); + null_map_holder = extractNestedColumnsAndNullMap(holder.key_columns, null_map); switch (data.type) { @@ -227,7 +226,7 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) break; #define M(NAME) \ case SetVariants::Type::NAME: \ - insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ + insertFromBlockImpl(*data.NAME, holder.key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \ break; APPLY_FOR_SET_VARIANTS(M) #undef M @@ -445,6 +444,11 @@ void Set::checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) c MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector && indexes_mapping_) : has_all_keys(set_elements.size() == indexes_mapping_.size()), indexes_mapping(std::move(indexes_mapping_)) { + // std::cerr << "MergeTreeSetIndex::MergeTreeSetIndex " + // << set_elements.size() << ' ' << indexes_mapping.size() << std::endl; + // for (const auto & vv : indexes_mapping) + // std::cerr << vv.key_index << ' ' << vv.tuple_index << std::endl; + ::sort(indexes_mapping.begin(), indexes_mapping.end(), [](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r) { @@ -487,6 +491,7 @@ MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector & key_ranges, const DataTypes & data_types, bool single_point) const { size_t tuple_size = indexes_mapping.size(); + // std::cerr << "MergeTreeSetIndex::checkInRange " << single_point << ' ' << tuple_size << ' ' << has_all_keys << std::endl; FieldValues left_point; FieldValues right_point; diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index cb47fde7f7d..c2931d79de0 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -53,7 +53,8 @@ public: { //ColumnRawPtrs key_columns; /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - Columns key_columns; + ColumnRawPtrs key_columns; + Columns materialized_columns; ColumnPtr null_map_holder; ColumnUInt8::MutablePtr filter; }; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index acd9147b613..4940cbd032c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1174,6 +1174,7 @@ static ActionsDAGPtr buildFilterDAG( static void buildIndexes( std::optional & indexes, ActionsDAGPtr filter_actions_dag, + const MergeTreeData & data, const ContextPtr & context, const SelectQueryInfo & query_info, const StorageMetadataPtr & metadata_snapshot) @@ -1196,7 +1197,7 @@ static void buildIndexes( context, primary_key_column_names, primary_key.expression, - array_join_name_set}, {}, false}); + array_join_name_set}, {}, {}, {}, false}); } else { @@ -1204,7 +1205,22 @@ static void buildIndexes( query_info, context, primary_key_column_names, - primary_key.expression}, {}, false}); + primary_key.expression}, {}, {}, {}, false}); + } + + if (metadata_snapshot->hasPartitionKey()) + { + const auto & partition_key = metadata_snapshot->getPartitionKey(); + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); + auto minmax_expression_actions = data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context)); + // minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); + + // if (context->getSettingsRef().allow_experimental_analyzer) + indexes->minmax_idx_condition.emplace(filter_actions_dag, context, minmax_columns_names, minmax_expression_actions, NameSet()); + // else + // indexes->minmax_idx_condition.emplace(query_info, context, minmax_columns_names, minmax_expression_actions); + + indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */); } indexes->use_skip_indexes = settings.use_skip_indexes; @@ -1250,7 +1266,7 @@ void ReadFromMergeTree::onAddFilterFinish() if (!filter_nodes.nodes.empty()) { auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info); - buildIndexes(indexes, filter_actions_dag, context, query_info, metadata_for_reading); + buildIndexes(indexes, filter_actions_dag, data, context, query_info, metadata_for_reading); } } @@ -1366,7 +1382,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( // } if (!indexes) - buildIndexes(indexes, query_info.filter_actions_dag, context, query_info, metadata_snapshot); + buildIndexes(indexes, query_info.filter_actions_dag, data, context, query_info, metadata_snapshot); if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue()) { @@ -1386,11 +1402,12 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( try { MergeTreeDataSelectExecutor::filterPartsByPartition( + indexes->partition_pruner, + indexes->minmax_idx_condition, parts, part_values, metadata_snapshot_base, data, - query_info, context, max_block_numbers_to_read.get(), log, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 6610b463726..5ff9d2f046c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -164,6 +165,8 @@ public: struct Indexes { KeyCondition key_condition; + std::optional partition_pruner; + std::optional minmax_idx_condition; UsefulSkipIndexes skip_indexes; bool use_skip_indexes; }; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7fcc111ced9..fe87198dcf5 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1157,6 +1157,8 @@ bool KeyCondition::tryPrepareSetIndex( RPNElement & out, size_t & out_key_column_num) { + // std::cerr << "::: tryPrepareSetIndex for " << func.getColumnName() << std::endl; + // std::cerr << StackTrace().toString() << std::endl; const auto & left_arg = func.getArgumentAt(0); out_key_column_num = 0; @@ -1200,7 +1202,10 @@ bool KeyCondition::tryPrepareSetIndex( } if (indexes_mapping.empty()) + { + // std::cerr << ".. index mapping is empty\n"; return false; + } const auto right_arg = func.getArgumentAt(1); @@ -1208,7 +1213,10 @@ bool KeyCondition::tryPrepareSetIndex( auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); if (!future_set) + { + // std::cerr << ".. no future set\n"; return false; + } // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); @@ -1220,13 +1228,21 @@ bool KeyCondition::tryPrepareSetIndex( auto prepared_set = future_set->get(); if (!prepared_set) + { + + // std::cerr << ".. no prepared set\n"; return false; + } // LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) + { + + // std::cerr << ".. no explicit elements\n"; return false; + } // LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); @@ -1235,7 +1251,7 @@ bool KeyCondition::tryPrepareSetIndex( prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]); out.set_index = std::make_shared(prepared_set->getSetElements(), std::move(indexes_mapping)); - + // std::cerr << ".. can use\n"; return true; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index f99e15c0fc1..f42fada4222 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -793,38 +793,28 @@ std::optional> MergeTreeDataSelectExecutor::filterPar } void MergeTreeDataSelectExecutor::filterPartsByPartition( + std::optional & partition_pruner, + std::optional & minmax_idx_condition, MergeTreeData::DataPartsVector & parts, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, - const SelectQueryInfo & query_info, const ContextPtr & context, const PartitionIdToMaxBlock * max_block_numbers_to_read, Poco::Logger * log, ReadFromMergeTree::IndexStats & index_stats) { const Settings & settings = context->getSettingsRef(); - - std::optional partition_pruner; - std::optional minmax_idx_condition; DataTypes minmax_columns_types; if (metadata_snapshot->hasPartitionKey()) { const auto & partition_key = metadata_snapshot->getPartitionKey(); - auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); - auto minmax_expression_actions = data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context)); minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); - if (context->getSettingsRef().allow_experimental_analyzer) - minmax_idx_condition.emplace(query_info.filter_actions_dag, context, minmax_columns_names, minmax_expression_actions, NameSet()); - else - minmax_idx_condition.emplace(query_info, context, minmax_columns_names, minmax_expression_actions); - - partition_pruner.emplace(metadata_snapshot, query_info, context, false /* strict */); - if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) { + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); throw Exception(ErrorCodes::INDEX_NOT_USED, "Neither MinMax index by columns ({}) nor partition expr is used and setting 'force_index_by_date' is set", fmt::join(minmax_columns_names, ", ")); @@ -1835,7 +1825,9 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( if (partition_pruner) { - if (partition_pruner->canBePruned(*part)) + auto val = partition_pruner->canBePruned(*part); + // std::cerr << "... part " << part->getNameWithState() << " cbp ? " << val << std::endl; + if (val) continue; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 8c8ce59bebe..f1efbdf0310 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -173,11 +173,12 @@ public: /// Filter parts using minmax index and partition key. static void filterPartsByPartition( + std::optional & partition_pruner, + std::optional & minmax_idx_condition, MergeTreeData::DataPartsVector & parts, const std::optional> & part_values, const StorageMetadataPtr & metadata_snapshot, const MergeTreeData & data, - const SelectQueryInfo & query_info, const ContextPtr & context, const PartitionIdToMaxBlock * max_block_numbers_to_read, Poco::Logger * log, diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 3d0883a1241..e2bf9bde674 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -31,12 +31,21 @@ namespace ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set) { + // std::cerr << "====== " << prepared_set->getDataTypes().size() << std::endl; if (prepared_set->getDataTypes().size() == 1) return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; Columns set_elements; for (auto & set_element : prepared_set->getSetElements()) + { + // std::cerr << set_element->dumpStructure() << std::endl; set_elements.emplace_back(set_element->convertToFullColumnIfConst()); + } + + // for (auto & set_element : prepared_set->getElementsTypes()) + // { + // // std::cerr << set_element->getName() << std::endl; + // } return {ColumnTuple::create(set_elements), std::make_shared(prepared_set->getElementsTypes()), "dummy"}; } @@ -331,6 +340,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (prepared_set && prepared_set->hasExplicitSetElements()) { const auto prepared_info = getPreparedSetInfo(prepared_set); + // std::cerr << "...... " << prepared_info.dumpStructure() << std::endl; if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) maybe_useful = true; } @@ -377,6 +387,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( size_t row_size = column->size(); size_t position = header.getPositionByName(key_node_column_name); const DataTypePtr & index_type = header.getByPosition(position).type; + // std::cerr << "::::: " << ColumnWithTypeAndName{column, type, ""}.dumpStructure() << " -> " << index_type->getName() << std::endl; const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index 35b2d5db3b5..a397a1475d1 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -24,6 +24,19 @@ PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, const Sele { } +PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAGPtr filter_actions_dag, ContextPtr context, bool strict) + : partition_key(MergeTreePartition::adjustPartitionKey(metadata, context)) + , partition_condition(filter_actions_dag, context, partition_key.column_names, partition_key.expression, {}, true /* single_point */, strict) + , useless(strict ? partition_condition.anyUnknownOrAlwaysTrue() : partition_condition.alwaysUnknownOrTrue()) +{ + // auto description = getKeyCondition().getDescription(); + // std::cerr << ".... " << description.condition << std::endl; + // std::cerr << filter_actions_dag->dumpDAG() << std::endl; + // for (const auto & name : partition_key.column_names) + // std::cerr << ". " << name << std::endl; + // std::cerr << partition_key.expression->dumpActions() << std::endl; +} + bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) { if (part.isEmpty()) @@ -39,6 +52,8 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) else { const auto & partition_value = part.partition.value; + // for (const auto & val : partition_value) + // std::cerr << val.dump() << std::endl; std::vector index_value(partition_value.begin(), partition_value.end()); for (auto & field : index_value) { @@ -49,6 +64,7 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) is_valid = partition_condition.mayBeTrueInRange( partition_value.size(), index_value.data(), index_value.data(), partition_key.data_types); + // std::cerr << "may be true " << is_valid << std::endl; partition_filter_map.emplace(partition_id, is_valid); if (!is_valid) diff --git a/src/Storages/MergeTree/PartitionPruner.h b/src/Storages/MergeTree/PartitionPruner.h index 3a986923321..7f1b74795c4 100644 --- a/src/Storages/MergeTree/PartitionPruner.h +++ b/src/Storages/MergeTree/PartitionPruner.h @@ -14,6 +14,7 @@ class PartitionPruner { public: PartitionPruner(const StorageMetadataPtr & metadata, const SelectQueryInfo & query_info, ContextPtr context, bool strict); + PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAGPtr filter_actions_dag, ContextPtr context, bool strict); bool canBePruned(const IMergeTreeDataPart & part); diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 34b463eadee..1e89427071c 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -161,6 +161,17 @@ public: } }; +/// Type of path to be fetched +enum class ZkPathType +{ + Exact, /// Fetch all nodes under this path + Prefix, /// Fetch all nodes starting with this prefix, recursively (multiple paths may match prefix) + Recurse, /// Fatch all nodes under this path, recursively +}; + +/// List of paths to be feched from zookeeper +using Paths = std::deque>; + class ReadFromSystemZooKeeper final : public SourceStepWithFilter { public: @@ -170,11 +181,14 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + void onAddFilterFinish() override; + private: - void fillData(MutableColumns & res_columns) const; + void fillData(MutableColumns & res_columns); std::shared_ptr storage_limits; ContextPtr context; + Paths paths; }; StorageSystemZooKeeper::StorageSystemZooKeeper(const StorageID & table_id_) @@ -246,17 +260,6 @@ NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes() }; } -/// Type of path to be fetched -enum class ZkPathType -{ - Exact, /// Fetch all nodes under this path - Prefix, /// Fetch all nodes starting with this prefix, recursively (multiple paths may match prefix) - Recurse, /// Fatch all nodes under this path, recursively -}; - -/// List of paths to be feched from zookeeper -using Paths = std::deque>; - static String pathCorrected(const String & path) { String path_corrected; @@ -421,10 +424,13 @@ static Paths extractPath(const ActionsDAG::NodeRawConstPtrs & filter_nodes, Cont } -void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns) const +void ReadFromSystemZooKeeper::onAddFilterFinish() { - Paths paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); + paths = extractPath(getFilterNodes().nodes, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); +} +void ReadFromSystemZooKeeper::fillData(MutableColumns & res_columns) +{ zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper(); if (paths.empty()) From e91934bceaea3809feb0e2e52532b9eeb35ad7f2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 May 2023 16:02:14 +0000 Subject: [PATCH 017/171] Fixing style. --- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 8d2f2204e84..fa4fc2c82df 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -202,7 +202,7 @@ public: external_storage_holder->future_set = std::move(future_set); } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "!!!!!!!!"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Set is already created for GLOBAL IN"); //prepared_sets->addStorageToSubquery(key, std::move(external_storage)); } From c7403284260992c296fbb34782bd1007a18ba28f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 30 May 2023 18:51:18 +0000 Subject: [PATCH 018/171] Fix more tests. --- src/Interpreters/ActionsVisitor.cpp | 11 +++++++---- src/Planner/CollectSets.cpp | 2 ++ src/Processors/QueryPlan/ReadFromMergeTree.cpp | 13 +++++++++++-- 3 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 23c6867a868..142b6f73b75 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1420,11 +1420,14 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return data.prepared_sets->addFromStorage(set_key, storage_set->getSet()); } - // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; - if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) + if (!data.getContext()->isGlobalContext()) { - external_table_set = tmp_table->future_set; - // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; + // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; + if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) + { + external_table_set = tmp_table->future_set; + // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; + } } } diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 19dc569ff0c..913cf1c6ee3 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -90,6 +90,8 @@ public: in_second_argument_node_type == QueryTreeNodeType::UNION) { auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + if (sets.getFuture(set_key)) + return; auto subquery_options = select_query_options.subquery(); Planner subquery_planner( diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 367d3dcb525..8483df797ef 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1236,6 +1236,15 @@ static void buildIndexes( if (!indexes->use_skip_indexes) return; + const SelectQueryInfo * info = &query_info; + std::optional info_copy; + if (settings.allow_experimental_analyzer) + { + info_copy.emplace(query_info); + info_copy->filter_actions_dag = filter_actions_dag; + info = &*info_copy; + } + UsefulSkipIndexes skip_indexes; using Key = std::pair; std::map merged; @@ -1249,14 +1258,14 @@ static void buildIndexes( if (inserted) { skip_indexes.merged_indices.emplace_back(); - skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot); } skip_indexes.merged_indices[it->second].addIndex(index_helper); } else { - auto condition = index_helper->createIndexCondition(query_info, context); + auto condition = index_helper->createIndexCondition(*info, context); if (!condition->alwaysUnknownOrTrue()) skip_indexes.useful_indices.emplace_back(index_helper, condition); } From 0d4ed32baca8eb2d897bcfd66eed0d04781af166 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 31 May 2023 11:25:33 -0400 Subject: [PATCH 019/171] better exception message --- src/Functions/FunctionsCodingIP.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsCodingIP.h b/src/Functions/FunctionsCodingIP.h index bd53fa7e043..9d090abb736 100644 --- a/src/Functions/FunctionsCodingIP.h +++ b/src/Functions/FunctionsCodingIP.h @@ -341,7 +341,11 @@ ColumnPtr convertIPv6ToIPv4(ColumnPtr column, const PaddedPODArray * null { if constexpr (exception_mode == IPStringToNumExceptionMode::Throw) { - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", column->getName()); + char addr[IPV6_MAX_TEXT_LENGTH + 1] {}; + char * paddr = addr; + formatIPv6(src, paddr); + + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 {} in column {} is not in IPv4 mapping block", addr, column->getName()); } else if constexpr (exception_mode == IPStringToNumExceptionMode::Default) { From 3af7e0a6fa21d570f78fcf9366c299e3199d2b77 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 31 May 2023 11:26:58 -0400 Subject: [PATCH 020/171] better exception message --- src/Functions/FunctionsConversion.h | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 4d4efc84df1..6d22fb661c3 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -310,7 +310,13 @@ struct ConvertImpl const uint8_t ip4_cidr[] {0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0xff, 0xff, 0x00, 0x00, 0x00, 0x00}; const uint8_t * src = reinterpret_cast(&vec_from[i].toUnderType()); if (!matchIPv6Subnet(src, ip4_cidr, 96)) - throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 in column {} is not in IPv4 mapping block", named_from.column->getName()); + { + char addr[IPV6_MAX_TEXT_LENGTH + 1] {}; + char * paddr = addr; + formatIPv6(src, paddr); + + throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "IPv6 {} in column {} is not in IPv4 mapping block", addr, named_from.column->getName()); + } uint8_t * dst = reinterpret_cast(&vec_to[i].toUnderType()); if constexpr (std::endian::native == std::endian::little) From 92859ebb3baad37e6538118f5e2c5c8016754b7f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 1 Jun 2023 21:15:15 +0000 Subject: [PATCH 021/171] Fixing more tests. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 +- src/Functions/in.cpp | 12 +- src/Interpreters/ActionsVisitor.cpp | 4 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Interpreters/PreparedSets.cpp | 119 ++++++++++-------- src/Interpreters/PreparedSets.h | 25 ++-- src/Interpreters/Set.cpp | 30 ++++- src/Interpreters/Set.h | 7 +- src/Planner/CollectSets.cpp | 2 +- .../CreateSetAndFilterOnTheFlyStep.cpp | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 8 +- src/Processors/QueryPlan/CreatingSetsStep.h | 6 +- .../Transforms/CreatingSetsTransform.cpp | 11 +- .../Transforms/CreatingSetsTransform.h | 6 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 10 +- src/QueryPipeline/QueryPipelineBuilder.h | 10 +- src/Storages/MergeTree/RPNBuilder.cpp | 53 ++++---- src/Storages/StorageSet.cpp | 4 +- .../01786_explain_merge_tree.reference | 4 +- 19 files changed, 195 insertions(+), 122 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 05b3a545dca..b39aff86d32 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5141,7 +5141,7 @@ 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, true /*fill_set_elements*/, settings.transform_null_in); + auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, 0, settings.transform_null_in); set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); set->insertFromBlock(result_block.getColumnsWithTypeAndName()); diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index 6a88a413c63..0e576b92aad 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -55,13 +55,9 @@ public: /// It is needed to perform type analysis without creation of set. static constexpr auto name = FunctionInName::name; - FunctionIn(SizeLimits size_limits_, bool transform_null_in_) - : size_limits(std::move(size_limits_)), transform_null_in(transform_null_in_) {} - - static FunctionPtr create(ContextPtr context) + static FunctionPtr create(ContextPtr) { - const auto & settings = context->getSettingsRef(); - return std::make_shared(FutureSet::getSizeLimitsForSet(settings, false), settings.transform_null_in); + return std::make_shared(); } String getName() const override @@ -182,10 +178,6 @@ public: return res; } - -private: - SizeLimits size_limits; - bool transform_null_in; }; template diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 142b6f73b75..59bbc74ca3a 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1435,7 +1435,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool // String set_id = right_in_operand->getColumnName(); //bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; SubqueryForSet subquery_for_set; // = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); - subquery_for_set.key = right_in_operand->getColumnName(); + subquery_for_set.key = set_key.toString(); //right_in_operand->getColumnName(); /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, @@ -1450,7 +1450,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool subquery_for_set.createSource(*interpreter); } - return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), std::move(external_table_set)); + return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), data.getContext()->getSettingsRef(), std::move(external_table_set)); } else { diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 624b33ea66b..cbdfb826f85 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -197,7 +197,7 @@ public: subquery_for_set.createSource(*interpreter); //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; - auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), getContext()->getSettingsRef(), nullptr); // std::cerr << "... Future set " << reinterpret_cast(external_storage_holder.get()) << " " << reinterpret_cast(future_set.get()) << std::endl; external_storage_holder->future_set = std::move(future_set); } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index c71273e0baa..1e475e8403d 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -132,10 +132,10 @@ FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, return it->second; } -FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set) +FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) { auto id = subquery.key; - auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set)); + auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings.transform_null_in); auto [it, inserted] = sets.emplace(key, from_subquery); if (!inserted) @@ -210,6 +210,8 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo { std::lock_guard lock(cache_mutex); + // std::cerr << "PreparedSetsCache::findOrPromiseToBuild " << key << "\n" << StackTrace().toString() << std::endl; + auto it = cache.find(key); if (it != cache.end()) { @@ -231,10 +233,10 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) if (!context->getSettingsRef().use_index_for_in_with_subqueries) return nullptr; - if (set) + if (subquery.set) { - if (set->hasExplicitSetElements()) - return set; + if (subquery.set->hasExplicitSetElements()) + return subquery.set; return nullptr; } @@ -242,7 +244,7 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) // std::cerr << "... external_table_set " << reinterpret_cast(external_table_set.get()) << std::endl; if (external_table_set) - return set = external_table_set->buildOrderedSetInplace(context); + return subquery.set = external_table_set->buildOrderedSetInplace(context); auto plan = buildPlan(context, true); if (!plan) @@ -255,37 +257,44 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) CompletedPipelineExecutor executor(pipeline); executor.execute(); - return set; + subquery.set->checkIsCreated(); + + return subquery.set; +} + +static SizeLimits getSizeLimitsForSet(const Settings & settings) +{ + return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); } std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) { - if (set) + if (subquery.set) return nullptr; // std::cerr << StackTrace().toString() << std::endl; - auto set_cache = context->getPreparedSetsCache(); - if (set_cache) - { - auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); - if (from_cache.index() == 0) - { - subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); - } - else - { - LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); - set = std::get<1>(from_cache).get(); - return nullptr; - } - } + // auto set_cache = context->getPreparedSetsCache(); + // if (set_cache) + // { + // auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); + // if (from_cache.index() == 0) + // { + // subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); + // } + // else + // { + // LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); + // set = std::get<1>(from_cache).get(); + // return nullptr; + // } + // } const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); + auto size_limits = getSizeLimitsForSet(settings); - subquery.set = set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + subquery.set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); auto plan = subquery.detachSource(); auto description = subquery.key; @@ -297,7 +306,8 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c auto creating_set = std::make_unique( plan->getCurrentDataStream(), description, - std::move(subquery), + subquery, + shared_from_this(), SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), context); creating_set->setStepDescription("Create set for subquery"); @@ -305,31 +315,25 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c return plan; } +// static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) +// { +// if (settings.use_index_for_in_with_subqueries_max_values && +// settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) +// return getSizeLimitsForUnorderedSet(settings); -static SizeLimits getSizeLimitsForUnorderedSet(const Settings & settings) -{ - return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); -} +// return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); +// } -static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) -{ - if (settings.use_index_for_in_with_subqueries_max_values && - settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) - return getSizeLimitsForUnorderedSet(settings); - - return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); -} - -SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) -{ - return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); -} +// SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) +// { +// return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); +// } FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) { bool create_ordered_set = false; - auto size_limits = getSizeLimitsForSet(settings, create_ordered_set); - set = std::make_shared(size_limits, create_ordered_set, settings.transform_null_in); + auto size_limits = getSizeLimitsForSet(settings); + set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); Columns columns; @@ -345,8 +349,16 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) //block(std::move(block_)) } -FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_) - : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)) {} +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_) + : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)), transform_null_in(transform_null_in_) {} + +DataTypes FutureSetFromSubquery::getTypes() const +{ + if (subquery.set) + return subquery.set->getElementsTypes(); + + return Set::getElementTypes(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName(), transform_null_in); +} FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} @@ -356,13 +368,14 @@ SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) return set; const auto & settings = context->getSettingsRef(); - auto limits = getSizeLimitsForSet(settings, true); + size_t max_values = settings.use_index_for_in_with_subqueries_max_values; + bool too_many_values = max_values && max_values < set->getTotalRowCount(); + if (!too_many_values) + { + set->initSetElements(); + set->appendSetElements(set_key_columns); + } - if (!limits.check(set->getTotalRowCount(), set->getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED)) - return nullptr; - - set->initSetElements(); - set->appendSetElements(set_key_columns); return set; } diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index c510e690e25..fa7f7069994 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -72,7 +72,9 @@ public: virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; virtual std::unique_ptr build(const ContextPtr & context) = 0; - static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); + virtual DataTypes getTypes() const = 0; + + // static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); }; using FutureSetPtr = std::shared_ptr; @@ -90,6 +92,8 @@ public: std::unique_ptr build(const ContextPtr &) override; + DataTypes getTypes() const override { return set->getElementsTypes(); } + /// void buildForTuple(SizeLimits size_limits, bool transform_null_in); private: @@ -129,14 +133,14 @@ public: std::unique_ptr source; }; -class FutureSetFromSubquery : public FutureSet +class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this { public: - FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_); + FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_); - bool isReady() const override { return set != nullptr; } + bool isReady() const override { return subquery.set != nullptr && subquery.set->isCreated(); } bool isFilled() const override { return isReady(); } - SetPtr get() const override { return set; } + SetPtr get() const override { return subquery.set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; @@ -145,12 +149,15 @@ public: return buildPlan(context, false); } - void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } + DataTypes getTypes() const override; + + // void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } private: - SetPtr set; + //SetPtr set; SubqueryForSet subquery; FutureSetPtr external_table_set; + bool transform_null_in; std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; @@ -169,6 +176,8 @@ public: return set->hasExplicitSetElements() ? set : nullptr; } + DataTypes getTypes() const override { return set->getElementsTypes(); } + std::unique_ptr build(const ContextPtr &) override { return nullptr; } private: @@ -240,7 +249,7 @@ public: FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, FutureSetPtr external_table_set); + FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); //void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index bd9cafc66eb..52e9a3a9bcb 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -103,6 +103,25 @@ void NO_INLINE Set::insertFromBlockImplCase( } +DataTypes Set::getElementTypes(const ColumnsWithTypeAndName & header, bool transform_null_in) +{ + DataTypes data_types; + data_types.reserve(header.size()); + + for (const auto & column : header) + { + data_types.push_back(column.type); + if (const auto * low_cardinality_type = typeid_cast(data_types.back().get())) + data_types.back() = low_cardinality_type->getDictionaryType(); + + if (!transform_null_in) + data_types.back() = removeNullable(data_types.back()); + } + + return data_types; +} + + void Set::setHeader(const ColumnsWithTypeAndName & header) { std::lock_guard lock(rwlock); @@ -190,7 +209,16 @@ bool Set::insertFromColumns(const Columns & columns) bool inserted = insertFromColumns(columns, holder); if (inserted && fill_set_elements) - appendSetElements(holder); + { + if (max_elements_to_fill && max_elements_to_fill < data.getTotalRowCount()) + { + /// Drop filled elementes + fill_set_elements = false; + set_elements.clear(); + } + else + appendSetElements(holder); + } return inserted; } diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index c2931d79de0..9b76ab30a1b 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -30,9 +30,9 @@ public: /// (that is useful only for checking that some value is in the set and may not store the original values), /// store all set elements in explicit form. /// This is needed for subsequent use for index. - Set(const SizeLimits & limits_, bool fill_set_elements_, bool transform_null_in_) + Set(const SizeLimits & limits_, bool fill_set_elements_, size_t max_elements_to_fill_, bool transform_null_in_) : log(&Poco::Logger::get("Set")), - limits(limits_), fill_set_elements(fill_set_elements_), transform_null_in(transform_null_in_) + limits(limits_), fill_set_elements(fill_set_elements_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_) { } @@ -90,6 +90,8 @@ public: bool areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const; void checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const; + static DataTypes getElementTypes(const ColumnsWithTypeAndName & header, bool transform_null_in); + private: size_t keys_size = 0; Sizes key_sizes; @@ -126,6 +128,7 @@ private: /// Do we need to additionally store all elements of the set in explicit form for subsequent use for index. bool fill_set_elements; + size_t max_elements_to_fill; /// If true, insert NULL values to set. bool transform_null_in; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 913cf1c6ee3..d264810645a 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -110,7 +110,7 @@ public: subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); /// TODO - sets.addFromSubquery(set_key, std::move(subquery_for_set), nullptr); + sets.addFromSubquery(set_key, std::move(subquery_for_set), settings, nullptr); //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); } diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index fe362f64b96..ebd58c3dc95 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -105,7 +105,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , column_names(column_names_) , max_rows_in_set(max_rows_in_set_) - , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, true)) + , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, 0, true)) , filtering_set(nullptr) , crosswise_connection(crosswise_connection_) , position(position_) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 38506412531..0909ee9f1eb 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -35,12 +35,14 @@ static ITransformingStep::Traits getTraits() CreatingSetStep::CreatingSetStep( const DataStream & input_stream_, String description_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, ContextPtr context_) : ITransformingStep(input_stream_, Block{}, getTraits()) , description(std::move(description_)) - , subquery_for_set(std::move(subquery_for_set_)) + , subquery_for_set(subquery_for_set_) + , set(std::move(set_)) , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { @@ -48,7 +50,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, context->getPreparedSetsCache()); + pipeline.addCreatingSetsTransform(getOutputStream().header, subquery_for_set, std::move(set), network_transfer_limits, context->getPreparedSetsCache()); } void CreatingSetStep::updateOutputStream() diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 244bb27ba78..2a6cb43c45a 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -15,7 +15,8 @@ public: CreatingSetStep( const DataStream & input_stream_, String description_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, ContextPtr context_); @@ -30,7 +31,8 @@ private: void updateOutputStream() override; String description; - SubqueryForSet subquery_for_set; + SubqueryForSet & subquery_for_set; + FutureSetPtr set; SizeLimits network_transfer_limits; ContextPtr context; }; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 24e17ff210f..f6e6f7a191e 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -25,11 +25,13 @@ CreatingSetsTransform::~CreatingSetsTransform() = default; CreatingSetsTransform::CreatingSetsTransform( Block in_header_, Block out_header_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, PreparedSetsCachePtr prepared_sets_cache_) : IAccumulatingTransform(std::move(in_header_), std::move(out_header_)) - , subquery(std::move(subquery_for_set_)) + , subquery(subquery_for_set_) + , set(std::move(set_)) , network_transfer_limits(std::move(network_transfer_limits_)) , prepared_sets_cache(std::move(prepared_sets_cache_)) { @@ -61,6 +63,7 @@ void CreatingSetsTransform::startSubquery() auto from_cache = prepared_sets_cache->findOrPromiseToBuild(subquery.key); if (from_cache.index() == 0) { + LOG_TRACE(log, "Building set, key: {}", subquery.key); promise_to_build = std::move(std::get<0>(from_cache)); } else @@ -74,8 +77,8 @@ void CreatingSetsTransform::startSubquery() continue; } - subquery.promise_to_fill_set.set_value(ready_set); - subquery.set.reset(); + //subquery.promise_to_fill_set.set_value(ready_set); + subquery.set = ready_set; //.reset(); done_with_set = true; set_from_cache = true; } diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index ef586b0655c..27c330bdbc3 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -29,7 +29,8 @@ public: CreatingSetsTransform( Block in_header_, Block out_header_, - SubqueryForSet subquery_for_set_, + SubqueryForSet & subquery_for_set_, + FutureSetPtr set_, SizeLimits network_transfer_limits_, PreparedSetsCachePtr prepared_sets_cache_); @@ -42,7 +43,8 @@ public: Chunk generate() override; private: - SubqueryForSet subquery; + SubqueryForSet & subquery; + FutureSetPtr set; std::optional> promise_to_build; QueryPipeline table_out; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 4898501b980..08d90899f4a 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -569,14 +569,20 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe return left; } -void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache) +void QueryPipelineBuilder::addCreatingSetsTransform( + const Block & res_header, + SubqueryForSet & subquery_for_set, + FutureSetPtr set, + const SizeLimits & limits, + PreparedSetsCachePtr prepared_sets_cache) { resize(1); auto transform = std::make_shared( getHeader(), res_header, - std::move(subquery_for_set), + subquery_for_set, + std::move(set), limits, std::move(prepared_sets_cache)); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 85b6f5c6772..da8443a7e33 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -33,6 +33,9 @@ class TableJoin; class QueryPipelineBuilder; using QueryPipelineBuilderPtr = std::unique_ptr; +class FutureSet; +using FutureSetPtr = std::shared_ptr; + class PreparedSetsCache; using PreparedSetsCachePtr = std::shared_ptr; @@ -141,7 +144,12 @@ public: /// This is used for CreatingSets. void addPipelineBefore(QueryPipelineBuilder pipeline); - void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache); + void addCreatingSetsTransform( + const Block & res_header, + SubqueryForSet & subquery_for_set, + FutureSetPtr set, + const SizeLimits & limits, + PreparedSetsCachePtr prepared_sets_cache); PipelineExecutorPtr execute(); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index cc7ec45be6a..c9feb533ea3 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -353,34 +353,34 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( { const auto & prepared_sets = getTreeContext().getPreparedSets(); + /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information + /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets + /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check + /// that the types it was prepared with are compatible with the types of the primary key. + auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) + { + assert(indexes_mapping.size() == data_types.size()); + + for (size_t i = 0; i < indexes_mapping.size(); ++i) + { + if (indexes_mapping[i].tuple_index >= set_types.size()) + return false; + + auto lhs = recursiveRemoveLowCardinality(data_types[i]); + auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); + + if (!lhs->equals(*rhs)) + return false; + } + + return true; + }; + if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); - /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information - /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets - /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check - /// that the types it was prepared with are compatible with the types of the primary key. - auto types_match = [&indexes_mapping, &data_types](const DataTypes & set_types) - { - assert(indexes_mapping.size() == data_types.size()); - - for (size_t i = 0; i < indexes_mapping.size(); ++i) - { - if (indexes_mapping[i].tuple_index >= set_types.size()) - return false; - - auto lhs = recursiveRemoveLowCardinality(data_types[i]); - auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); - - if (!lhs->equals(*rhs)) - return false; - } - - return true; - }; - auto tree_hash = ast_node->getTreeHash(); for (const auto & [key, future_set] : prepared_sets->getSets()) { @@ -392,7 +392,12 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( { const auto * node_without_alias = getNodeWithoutAlias(dag_node); if (node_without_alias->column) - return tryGetSetFromDAGNode(node_without_alias); + { + auto future_set = tryGetSetFromDAGNode(node_without_alias); + auto set_types = future_set->getTypes(); + if (types_match(future_set->getTypes())) + return future_set; + } } return nullptr; diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index 00b5dbfc5e3..da94c87b69e 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -147,7 +147,7 @@ StorageSet::StorageSet( const String & comment, bool persistent_) : StorageSetOrJoinBase{disk_, relative_path_, table_id_, columns_, constraints_, comment, persistent_} - , set(std::make_shared(SizeLimits(), false, true)) + , set(std::make_shared(SizeLimits(), false, 0, true)) { Block header = getInMemoryMetadataPtr()->getSampleBlock(); set->setHeader(header.getColumnsWithTypeAndName()); @@ -176,7 +176,7 @@ void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn Block header = metadata_snapshot->getSampleBlock(); increment = 0; - set = std::make_shared(SizeLimits(), false, true); + set = std::make_shared(SizeLimits(), false, 0, true); set->setHeader(header.getColumnsWithTypeAndName()); } diff --git a/tests/queries/0_stateless/01786_explain_merge_tree.reference b/tests/queries/0_stateless/01786_explain_merge_tree.reference index e6628813dbd..8d3954484dd 100644 --- a/tests/queries/0_stateless/01786_explain_merge_tree.reference +++ b/tests/queries/0_stateless/01786_explain_merge_tree.reference @@ -10,7 +10,7 @@ Keys: y bitAnd(z, 3) - Condition: and((bitAnd(z, 3) not in [1, 1]), and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))) + Condition: and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])) Parts: 3/4 Granules: 10/11 PrimaryKey @@ -46,7 +46,7 @@ { "Type": "Partition", "Keys": ["y", "bitAnd(z, 3)"], - "Condition": "and((bitAnd(z, 3) not in [1, 1]), and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1])))", + "Condition": "and((y in [1, +Inf)), (bitAnd(z, 3) not in [1, 1]))", "Initial Parts": 4, "Selected Parts": 3, "Initial Granules": 11, From 36129c3d20bef74c4502126b2c37001e2c80b8d2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Jun 2023 13:36:14 +0000 Subject: [PATCH 022/171] Try to fix last tests. --- src/Storages/MergeTree/RPNBuilder.cpp | 6 ++++-- .../0_stateless/00981_in_subquery_with_tuple.reference | 3 --- tests/queries/0_stateless/01651_bugs_from_15889.reference | 1 + tests/queries/0_stateless/01651_bugs_from_15889.sql | 5 ++--- 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index c9feb533ea3..b82c350fa1a 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -366,8 +366,10 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (indexes_mapping[i].tuple_index >= set_types.size()) return false; - auto lhs = recursiveRemoveLowCardinality(data_types[i]); - auto rhs = recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index]); + auto lhs = removeNullable(recursiveRemoveLowCardinality(data_types[i])); + auto rhs = removeNullable(recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index])); + + // std::cerr << "============ " << lhs->getName() << ' ' << rhs->getName() << std::endl; if (!lhs->equals(*rhs)) return false; diff --git a/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference b/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference index 673d035ede6..833a8c93d24 100644 --- a/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference +++ b/tests/queries/0_stateless/00981_in_subquery_with_tuple.reference @@ -1,7 +1,4 @@ -OK1 OK2 OK3 -OK4 -OK5 2019-08-11 world 2019-08-12 hello diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.reference b/tests/queries/0_stateless/01651_bugs_from_15889.reference index 8b137891791..77ac542d4fb 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.reference +++ b/tests/queries/0_stateless/01651_bugs_from_15889.reference @@ -1 +1,2 @@ +0 diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index 4717a8dcc0d..6a8c6d35911 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -8,10 +8,9 @@ INSERT INTO xp SELECT '2020-01-01', number, '' FROM numbers(100000); CREATE TABLE xp_d AS xp ENGINE = Distributed(test_shard_localhost, currentDatabase(), xp); --- FIXME: this query spontaneously returns either 8 or 20 error code. Looks like it's potentially flaky. --- SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 8 } +SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 8 } -SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); -- { serverError 53 } +SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); DROP TABLE IF EXISTS xp; DROP TABLE IF EXISTS xp_d; From 975e58c56d5cf7038052e0be9699c094c7203161 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Jun 2023 15:15:44 +0000 Subject: [PATCH 023/171] Fix style. --- src/Interpreters/PreparedSets.cpp | 1 - src/Planner/CollectSets.cpp | 2 ++ src/Processors/QueryPlan/CreatingSetsStep.cpp | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 1e475e8403d..6df7c748e60 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int SET_SIZE_LIMIT_EXCEEDED; } PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index d264810645a..5f44994c14b 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -81,6 +81,8 @@ public: element_type = low_cardinality_type->getDictionaryType(); auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + if (sets.getFuture(set_key)) + return; sets.addFromTuple(set_key, std::move(set), settings); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 0909ee9f1eb..264c4b9ef47 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -46,6 +46,7 @@ CreatingSetStep::CreatingSetStep( , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { + std::cerr << StackTrace().toString() << std::endl; } void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) From c19866f72d3b356f2c4fed334c2beeecd441ef0e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 2 Jun 2023 15:16:36 +0000 Subject: [PATCH 024/171] Fix style. --- src/Processors/QueryPlan/CreatingSetsStep.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 264c4b9ef47..0909ee9f1eb 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -46,7 +46,6 @@ CreatingSetStep::CreatingSetStep( , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { - std::cerr << StackTrace().toString() << std::endl; } void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) From 34c4b89b161df4427912d402fa6aee4a6821dde1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 4 Jun 2023 16:23:34 +0000 Subject: [PATCH 025/171] fix backward compatibility for IP types hashing in aggregate functions --- base/base/IPv4andIPv6.h | 24 +++++++++++++------ .../AggregateFunctionMap.cpp | 3 +++ src/AggregateFunctions/AggregateFunctionMap.h | 3 +++ .../AggregateFunctionUniq.cpp | 9 +++++++ .../AggregateFunctionUniq.h | 12 +++++----- src/Core/Types_fwd.h | 2 +- src/DataTypes/DataTypeMap.cpp | 3 ++- 7 files changed, 41 insertions(+), 15 deletions(-) diff --git a/base/base/IPv4andIPv6.h b/base/base/IPv4andIPv6.h index 7b745ec7b84..4aee2329572 100644 --- a/base/base/IPv4andIPv6.h +++ b/base/base/IPv4andIPv6.h @@ -7,16 +7,17 @@ namespace DB { - using IPv4 = StrongTypedef; + struct IPv4 : StrongTypedef + { + using StrongTypedef::StrongTypedef; + using StrongTypedef::operator=; + constexpr explicit IPv4(UInt64 value): StrongTypedef(static_cast(value)) {} + }; struct IPv6 : StrongTypedef { - constexpr IPv6() = default; - constexpr explicit IPv6(const UInt128 & x) : StrongTypedef(x) {} - constexpr explicit IPv6(UInt128 && x) : StrongTypedef(std::move(x)) {} - - IPv6 & operator=(const UInt128 & rhs) { StrongTypedef::operator=(rhs); return *this; } - IPv6 & operator=(UInt128 && rhs) { StrongTypedef::operator=(std::move(rhs)); return *this; } + using StrongTypedef::StrongTypedef; + using StrongTypedef::operator=; bool operator<(const IPv6 & rhs) const { @@ -62,4 +63,13 @@ namespace std return std::hash()(x.toUnderType()); } }; + + template <> + struct hash + { + size_t operator()(const DB::IPv4 & x) const + { + return std::hash()(x.toUnderType()); + } + }; } diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index f6100602f3f..38e4f49d9a2 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -100,6 +100,9 @@ public: return std::make_shared>(nested_function, arguments); case TypeIndex::UUID: return std::make_shared>(nested_function, arguments); + case TypeIndex::IPv4: + return std::make_shared>(nested_function, arguments); + case TypeIndex::IPv6: case TypeIndex::FixedString: case TypeIndex::String: return std::make_shared>(nested_function, arguments); diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 55f6611974e..4a4ae92735b 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -19,6 +19,7 @@ #include #include #include "DataTypes/Serializations/ISerialization.h" +#include #include "base/types.h" #include #include "AggregateFunctions/AggregateFunctionFactory.h" @@ -147,6 +148,8 @@ public: StringRef key_ref; if (key_type->getTypeId() == TypeIndex::FixedString) key_ref = assert_cast(key_column).getDataAt(offset + i); + else if (key_type->getTypeId() == TypeIndex::IPv6) + key_ref = assert_cast(key_column).getDataAt(offset + i); else key_ref = assert_cast(key_column).getDataAt(offset + i); diff --git a/src/AggregateFunctions/AggregateFunctionUniq.cpp b/src/AggregateFunctions/AggregateFunctionUniq.cpp index bf998c5ee9e..f5147daa97b 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -60,6 +61,10 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument return std::make_shared>(argument_types); else if (which.isUUID()) return std::make_shared>(argument_types); + else if (which.isIPv4()) + return std::make_shared>(argument_types); + else if (which.isIPv6()) + return std::make_shared>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) @@ -109,6 +114,10 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument return std::make_shared>>(argument_types); else if (which.isUUID()) return std::make_shared>>(argument_types); + else if (which.isIPv4()) + return std::make_shared>>(argument_types); + else if (which.isIPv6()) + return std::make_shared>>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index c782b9314fd..0524dd53ec0 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -248,17 +248,17 @@ struct Adder AggregateFunctionUniqUniquesHashSetData> || std::is_same_v>) { const auto & column = *columns[0]; - if constexpr (!std::is_same_v) + if constexpr (std::is_same_v || std::is_same_v) + { + StringRef value = column.getDataAt(row_num); + data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); + } + else { using ValueType = typename decltype(data.set)::value_type; const auto & value = assert_cast &>(column).getElement(row_num); data.set.insert(static_cast(AggregateFunctionUniqTraits::hash(value))); } - else - { - StringRef value = column.getDataAt(row_num); - data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); - } } else if constexpr (std::is_same_v>) { diff --git a/src/Core/Types_fwd.h b/src/Core/Types_fwd.h index 3db1127fe5a..a59e4b6eab8 100644 --- a/src/Core/Types_fwd.h +++ b/src/Core/Types_fwd.h @@ -27,7 +27,7 @@ namespace DB using UUID = StrongTypedef; -using IPv4 = StrongTypedef; +struct IPv4; struct IPv6; diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index 0f5d97e6761..90561857fad 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -69,7 +69,7 @@ void DataTypeMap::assertKeyType() const if (!checkKeyType(key_type)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type of Map key must be a type, that can be represented by integer " - "or String or FixedString (possibly LowCardinality) or UUID," + "or String or FixedString (possibly LowCardinality) or UUID or IPv6," " but {} given", key_type->getName()); } @@ -120,6 +120,7 @@ bool DataTypeMap::checkKeyType(DataTypePtr key_type) else if (!key_type->isValueRepresentedByInteger() && !isStringOrFixedString(*key_type) && !WhichDataType(key_type).isNothing() + && !WhichDataType(key_type).isIPv6() && !WhichDataType(key_type).isUUID()) { return false; From 760483d8d3a012b4b6456b9bdf14afd2d052a514 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 6 Jun 2023 04:18:51 +0000 Subject: [PATCH 026/171] multiple fixes --- base/base/IPv4andIPv6.h | 4 ++- .../AggregateFunctionMap.cpp | 1 + src/AggregateFunctions/AggregateFunctionMap.h | 26 +++++++++++++++++++ .../AggregateFunctionUniq.cpp | 2 +- .../AggregateFunctionUniq.h | 12 +++++++++ src/IO/ReadHelpers.h | 14 ++++++++++ src/IO/WriteHelpers.h | 8 ++++++ 7 files changed, 65 insertions(+), 2 deletions(-) diff --git a/base/base/IPv4andIPv6.h b/base/base/IPv4andIPv6.h index 4aee2329572..e2f93b54124 100644 --- a/base/base/IPv4andIPv6.h +++ b/base/base/IPv4andIPv6.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -55,12 +56,13 @@ namespace DB namespace std { + /// For historical reasons we hash IPv6 as a FixedString(16) template <> struct hash { size_t operator()(const DB::IPv6 & x) const { - return std::hash()(x.toUnderType()); + return std::hash{}(std::string_view(reinterpret_cast(&x.toUnderType()), IPV6_BINARY_LENGTH)); } }; diff --git a/src/AggregateFunctions/AggregateFunctionMap.cpp b/src/AggregateFunctions/AggregateFunctionMap.cpp index 38e4f49d9a2..b957b541fe1 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.cpp +++ b/src/AggregateFunctions/AggregateFunctionMap.cpp @@ -103,6 +103,7 @@ public: case TypeIndex::IPv4: return std::make_shared>(nested_function, arguments); case TypeIndex::IPv6: + return std::make_shared>(nested_function, arguments); case TypeIndex::FixedString: case TypeIndex::String: return std::make_shared>(nested_function, arguments); diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index 4a4ae92735b..7b9bb088d8f 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -21,6 +21,7 @@ #include "DataTypes/Serializations/ISerialization.h" #include #include "base/types.h" +#include #include #include "AggregateFunctions/AggregateFunctionFactory.h" @@ -70,6 +71,31 @@ struct AggregateFunctionMapCombinatorData } }; +/// Specialization for IPv6 - for historical reasons it should be stored as FixedString(16) +template <> +struct AggregateFunctionMapCombinatorData +{ + struct IPv6Hash + { + using hash_type = std::hash; + using is_transparent = void; + + size_t operator()(const IPv6 & ip) const { return hash_type{}(ip); } + }; + + using SearchType = IPv6; + std::unordered_map> merged_maps; + + static void writeKey(const IPv6 & key, WriteBuffer & buf) + { + writeIPv6Binary(key, buf); + } + static void readKey(IPv6 & key, ReadBuffer & buf) + { + readIPv6Binary(key, buf); + } +}; + template class AggregateFunctionMap final : public IAggregateFunctionDataHelper, AggregateFunctionMap> diff --git a/src/AggregateFunctions/AggregateFunctionUniq.cpp b/src/AggregateFunctions/AggregateFunctionUniq.cpp index f5147daa97b..748a232641e 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -117,7 +117,7 @@ createAggregateFunctionUniq(const std::string & name, const DataTypes & argument else if (which.isIPv4()) return std::make_shared>>(argument_types); else if (which.isIPv6()) - return std::make_shared>>(argument_types); + return std::make_shared>>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index 0524dd53ec0..03d999b47e2 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -101,6 +101,18 @@ struct AggregateFunctionUniqHLL12Data static String getName() { return "uniqHLL12"; } }; +template <> +struct AggregateFunctionUniqHLL12Data +{ + using Set = HyperLogLogWithSmallSetOptimization; + Set set; + + constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_variadic = false; + + static String getName() { return "uniqHLL12"; } +}; + template struct AggregateFunctionUniqHLL12DataForVariadic { diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 32338552b66..1aa294f76bf 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -61,6 +61,7 @@ namespace ErrorCodes extern const int INCORRECT_DATA; extern const int TOO_LARGE_STRING_SIZE; extern const int TOO_LARGE_ARRAY_SIZE; + extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH; } /// Helper functions for formatted input. @@ -136,6 +137,19 @@ inline void readStringBinary(std::string & s, ReadBuffer & buf, size_t max_strin buf.readStrict(s.data(), size); } +/// For historical reasons we store IPv6 as a String +inline void readIPv6Binary(IPv6 & ip, ReadBuffer & buf) +{ + size_t size = 0; + readVarUInt(size, buf); + + if (size != IPV6_BINARY_LENGTH) + throw Exception(ErrorCodes::SIZE_OF_FIXED_STRING_DOESNT_MATCH, + "Size of the string {} doesn't match size of binary IPv6 {}", size, IPV6_BINARY_LENGTH); + + buf.readStrict(reinterpret_cast(&ip.toUnderType()), size); +} + template void readVectorBinary(std::vector & v, ReadBuffer & buf) { diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index cdbc952690c..505a2f988f0 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -10,6 +10,7 @@ #include +#include "Common/formatIPv6.h" #include #include #include @@ -104,6 +105,13 @@ inline void writeStringBinary(const std::string & s, WriteBuffer & buf) buf.write(s.data(), s.size()); } +/// For historical reasons we store IPv6 as a String +inline void writeIPv6Binary(const IPv6 & ip, WriteBuffer & buf) +{ + writeVarUInt(IPV6_BINARY_LENGTH, buf); + buf.write(reinterpret_cast(&ip.toUnderType()), IPV6_BINARY_LENGTH); +} + inline void writeStringBinary(StringRef s, WriteBuffer & buf) { writeVarUInt(s.size, buf); From f4bf42cc42a6a2d8966d711874286427956d51d7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 8 Jun 2023 17:29:51 +0000 Subject: [PATCH 027/171] more fixes --- .../AggregateFunctionGroupArray.cpp | 1 + .../AggregateFunctionGroupUniqArray.cpp | 11 ++ .../AggregateFunctionTopK.cpp | 19 ++ .../AggregateFunctionUniq.h | 31 +++- .../AggregateFunctionUniqCombined.cpp | 5 + .../AggregateFunctionUniqCombined.h | 16 +- ...es_aggregate_functions_states.reference.j2 | 172 ++++++++++++++++++ ...ip_types_aggregate_functions_states.sql.j2 | 123 +++++++++++++ 8 files changed, 366 insertions(+), 12 deletions(-) create mode 100644 tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 create mode 100644 tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index 15f500b8bb6..bb1368b9ff8 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -25,6 +25,7 @@ IAggregateFunction * createWithNumericOrTimeType(const IDataType & argument_type WhichDataType which(argument_type); if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate(std::forward(args)...); if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate(std::forward(args)...); + if (which.idx == TypeIndex::IPv4) return new AggregateFunctionTemplate(std::forward(args)...); return createWithNumericType(argument_type, std::forward(args)...); } diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp index 16f2feb71bf..9e8060d44cc 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -39,12 +40,22 @@ public: static DataTypePtr createResultType() { return std::make_shared(std::make_shared()); } }; +template +class AggregateFunctionGroupUniqArrayIPv4 : public AggregateFunctionGroupUniqArray +{ +public: + explicit AggregateFunctionGroupUniqArrayIPv4(const DataTypePtr & argument_type, const Array & parameters_, UInt64 max_elems_ = std::numeric_limits::max()) + : AggregateFunctionGroupUniqArray(argument_type, parameters_, createResultType(), max_elems_) {} + static DataTypePtr createResultType() { return std::make_shared(std::make_shared()); } +}; + template IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, TArgs && ... args) { WhichDataType which(argument_type); if (which.idx == TypeIndex::Date) return new AggregateFunctionGroupUniqArrayDate(argument_type, std::forward(args)...); else if (which.idx == TypeIndex::DateTime) return new AggregateFunctionGroupUniqArrayDateTime(argument_type, std::forward(args)...); + else if (which.idx == TypeIndex::IPv4) return new AggregateFunctionGroupUniqArrayIPv4(argument_type, std::forward(args)...); else { /// Check that we can use plain version of AggregateFunctionGroupUniqArrayGeneric diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index e568694df02..8f6652223cc 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -5,6 +5,7 @@ #include #include #include +#include static inline constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF; @@ -60,6 +61,22 @@ public: {} }; +template +class AggregateFunctionTopKIPv4 : public AggregateFunctionTopK +{ +public: + using AggregateFunctionTopK::AggregateFunctionTopK; + + AggregateFunctionTopKIPv4(UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params) + : AggregateFunctionTopK( + threshold_, + load_factor, + argument_types_, + params, + std::make_shared(std::make_shared())) + {} +}; + template IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt64 threshold, UInt64 load_factor, const Array & params) @@ -72,6 +89,8 @@ IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt return new AggregateFunctionTopKDate(threshold, load_factor, argument_types, params); if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTopKDateTime(threshold, load_factor, argument_types, params); + if (which.idx == TypeIndex::IPv4) + return new AggregateFunctionTopKIPv4(threshold, load_factor, argument_types, params); /// Check that we can use plain version of AggregateFunctionTopKGeneric if (argument_types[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index 03d999b47e2..de68e9076a0 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -167,6 +167,25 @@ struct AggregateFunctionUniqExactData static String getName() { return "uniqExact"; } }; +/// For historical reasons IPv6 is treated as FixedString(16) +template +struct AggregateFunctionUniqExactData +{ + using Key = UInt128; + + /// When creating, the hash table must be small. + using SingleLevelSet = HashSet, HashTableAllocatorWithStackMemory>; + using TwoLevelSet = TwoLevelHashSet; + using Set = UniqExactSet; + + Set set; + + constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_variadic = false; + + static String getName() { return "uniqExact"; } +}; + template struct AggregateFunctionUniqExactDataForVariadic : AggregateFunctionUniqExactData { @@ -275,12 +294,7 @@ struct Adder else if constexpr (std::is_same_v>) { const auto & column = *columns[0]; - if constexpr (!std::is_same_v) - { - data.set.template insert( - assert_cast &>(column).getData()[row_num]); - } - else + if constexpr (std::is_same_v || std::is_same_v) { StringRef value = column.getDataAt(row_num); @@ -291,6 +305,11 @@ struct Adder data.set.template insert(key); } + else + { + data.set.template insert( + assert_cast &>(column).getData()[row_num]); + } } #if USE_DATASKETCHES else if constexpr (std::is_same_v) diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index 1c59da59e83..8c2cb6ea0de 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include @@ -60,6 +61,10 @@ namespace return std::make_shared::template AggregateFunction>(argument_types, params); else if (which.isUUID()) return std::make_shared::template AggregateFunction>(argument_types, params); + else if (which.isIPv4()) + return std::make_shared::template AggregateFunction>(argument_types, params); + else if (which.isIPv6()) + return std::make_shared::template AggregateFunction>(argument_types, params); else if (which.isTuple()) { if (use_exact_hash_function) diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/src/AggregateFunctions/AggregateFunctionUniqCombined.h index d879e3b3dde..5e8fa69f9de 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -119,6 +119,10 @@ struct AggregateFunctionUniqCombinedData : public Aggr { }; +template +struct AggregateFunctionUniqCombinedData : public AggregateFunctionUniqCombinedDataWithKey +{ +}; template class AggregateFunctionUniqCombined final @@ -141,16 +145,16 @@ public: void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { - if constexpr (!std::is_same_v) - { - const auto & value = assert_cast &>(*columns[0]).getElement(row_num); - this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits::hash(value)); - } - else + if constexpr (std::is_same_v || std::is_same_v) { StringRef value = columns[0]->getDataAt(row_num); this->data(place).set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); } + else + { + const auto & value = assert_cast &>(*columns[0]).getElement(row_num); + this->data(place).set.insert(detail::AggregateFunctionUniqCombinedTraits::hash(value)); + } } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 new file mode 100644 index 00000000000..481dd723b66 --- /dev/null +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 @@ -0,0 +1,172 @@ +----- hash / State / ip4 ----- +Row 1: +────── +minState: 12535288824949910799 +maxState: 18210943739258811465 +first_valueState: 12535288824949910799 +last_valueState: 18210943739258811465 +topKState: 1594227852744382511 +groupArrayState: 8025417272361615478 +groupUniqArrayState: 919082878249747568 +uniqState: 14828781561416784358 +uniqExactState: 11391659146320471795 +uniqCombinedState: 9631896280254268221 +uniqCombined64State: 5156097536649078816 +uniqHLL12State: 9696624347265201099 +uniqThetaState: 10464560810701154023 +----- hash / State / ip6 ----- +Row 1: +────── +minState: 9428555662807296659 +maxState: 18253481702148601156 +first_valueState: 9428555662807296659 +last_valueState: 18253481702148601156 +topKState: 8045294331733869941 +groupArrayState: 10451014709837753966 +groupUniqArrayState: 1954028114836070615 +uniqState: 14986562136250471284 +uniqExactState: 10032843621916709112 +uniqCombinedState: 6379274083567016598 +uniqCombined64State: 6379274083567016598 +uniqHLL12State: 9181286681186915812 +uniqThetaState: 2415188383468008881 +----- finalizeAggregation / State / ip4 ----- +Row 1: +────── +min: 59.154.201.255 +max: 59.154.202.48 +first_value: 59.154.201.255 +last_value: 59.154.202.48 +topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] +groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 49 +uniqTheta: 50 +----- finalizeAggregation / State / ip6 ----- +Row 1: +────── +min: 8c:333c::8c:333c:0:0 +max: ff8b:333c::ff8b:333c:0:0 +first_value: 8c:333c::8c:333c:0:0 +last_value: ff8b:333c::ff8b:333c:0:0 +topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 50 +uniqTheta: 50 +----- hash / IfState / ip4 ----- +Row 1: +────── +minIfState: 12535288824949910799 +maxIfState: 18210943739258811465 +first_valueIfState: 12535288824949910799 +last_valueIfState: 18210943739258811465 +topKIfState: 1594227852744382511 +groupArrayIfState: 8025417272361615478 +groupUniqArrayIfState: 919082878249747568 +uniqIfState: 14828781561416784358 +uniqExactIfState: 11391659146320471795 +uniqCombinedIfState: 9631896280254268221 +uniqCombined64IfState: 5156097536649078816 +uniqHLL12IfState: 9696624347265201099 +uniqThetaIfState: 10464560810701154023 +----- hash / IfState / ip6 ----- +Row 1: +────── +minIfState: 9428555662807296659 +maxIfState: 18253481702148601156 +first_valueIfState: 9428555662807296659 +last_valueIfState: 18253481702148601156 +topKIfState: 8045294331733869941 +groupArrayIfState: 10451014709837753966 +groupUniqArrayIfState: 1954028114836070615 +uniqIfState: 14986562136250471284 +uniqExactIfState: 10032843621916709112 +uniqCombinedIfState: 6379274083567016598 +uniqCombined64IfState: 6379274083567016598 +uniqHLL12IfState: 9181286681186915812 +uniqThetaIfState: 2415188383468008881 +----- finalizeAggregation / IfState / ip4 ----- +Row 1: +────── +min: 59.154.201.255 +max: 59.154.202.48 +first_value: 59.154.201.255 +last_value: 59.154.202.48 +topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] +groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 49 +uniqTheta: 50 +----- finalizeAggregation / IfState / ip6 ----- +Row 1: +────── +min: 8c:333c::8c:333c:0:0 +max: ff8b:333c::ff8b:333c:0:0 +first_value: 8c:333c::8c:333c:0:0 +last_value: ff8b:333c::ff8b:333c:0:0 +topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 50 +uniqExact: 50 +uniqCombined: 50 +uniqCombined64: 50 +uniqHLL12: 50 +uniqTheta: 50 +----- Arg / hash / State / ip4 ----- +Row 1: +────── +argMinState: 13774589216353164344 +argMaxState: 9177365218111013695 +----- Arg / hash / State / ip6 ----- +Row 1: +────── +argMinState: 7320668278649617037 +argMaxState: 16598449636475438091 +----- Arg / finalizeAggregation / State / ip4 ----- +Row 1: +────── +argMinState: 59.154.201.255 +argMaxState: 59.154.202.48 +----- Arg / finalizeAggregation / State / ip6 ----- +Row 1: +────── +argMinState: 8c:333c::8c:333c:0:0 +argMaxState: ff8b:333c::ff8b:333c:0:0 +----- hash / State / ip4 ----- +Row 1: +────── +anyState: 12535288824949910799 +anyHeavyState: 9327034461443333306 +anyLastState: 12535288824949910799 +----- hash / State / ip6 ----- +Row 1: +────── +anyState: 1383994153676807399 +anyHeavyState: 15555709096566410627 +anyLastState: 1383994153676807399 +----- finalizeAggregation / State / ip4 ----- +Row 1: +────── +any: 59.154.201.255 +anyHeavy: 59.154.201.255 +anyLast: 59.154.201.255 +----- finalizeAggregation / State / ip6 ----- +Row 1: +────── +any: dd8b:333c::dd8b:333c:0:0 +anyHeavy: dd8b:333c::dd8b:333c:0:0 +anyLast: dd8b:333c::dd8b:333c:0:0 diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 new file mode 100644 index 00000000000..133d5287fdb --- /dev/null +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -0,0 +1,123 @@ +{# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} + +{% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,50) order by ip" %} +{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 50) order by ip" %} +{% set ip_generators = {'ip4': ip4_generator, 'ip6': ip6_generator} %} + + +{% set agg_func_list = [ "min", "max", "first_value", "last_value", "topK", "groupArray", "groupUniqArray", "uniq", "uniqExact", "uniqCombined", "uniqCombined64", "uniqHLL12", "uniqTheta" ] %} + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(ip) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(ip) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- hash / IfState / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}IfState(ip, 1) )) AS {{ func }}IfState{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- finalizeAggregation / IfState / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}IfState(ip, 1) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + + +{% set agg_func_list = [ "argMin", "argMax" ] %} + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Arg / hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(ip, ip) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Arg / finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(ip, ip) ) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + +{# let's test functions with not deterministic result against 1 row, to make it deterministic #} +{% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,1) order by ip" %} +{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 1) order by ip" %} + +{% set ip_generators = {'ip4': ip4_generator, 'ip6': ip6_generator} %} + +{% set agg_func_list = [ "any", "anyHeavy", "anyLast" ] %} + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(ip) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(ip) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + From c96989ca14415707e5dc37958ca36093e8292f46 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 8 Jun 2023 19:37:52 +0000 Subject: [PATCH 028/171] no-fasttest because uniqTheta --- .../02751_ip_types_aggregate_functions_states.sql.j2 | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 index 133d5287fdb..708eeab7724 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- no-fasttest because uniqTheta + {# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} {% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,50) order by ip" %} From 9a4043a4b4c97bcfb7eb345e0753b27228c2f4f7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Jun 2023 17:51:59 +0000 Subject: [PATCH 029/171] Fixing more tests. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 1 - src/Interpreters/PreparedSets.h | 6 ++ src/Planner/CollectSets.cpp | 25 +++--- src/Planner/CollectSets.h | 2 +- src/Planner/Planner.cpp | 103 +++++++++++----------- src/Planner/Utils.cpp | 4 +- src/Storages/StorageDistributed.cpp | 11 ++- 7 files changed, 82 insertions(+), 70 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index b39aff86d32..bab64480901 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2333,7 +2333,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout); auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context); - auto result = std::make_shared(std::move(storage), std::move(storage_lock), std::move(storage_snapshot)); if (is_temporary_table) result->setTemporaryTableName(table_name); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index fa7f7069994..a119c24ad10 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -29,6 +29,9 @@ class Set; using SetPtr = std::shared_ptr; class InterpreterSelectWithUnionQuery; +class IQueryTreeNode; +using QueryTreeNodePtr = std::shared_ptr; + /// Represents a set in a query that might be referenced at analysis time and built later during execution. /// Also it can represent a constant set that is ready to use. /// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there @@ -131,6 +134,7 @@ public: /// The source is obtained using the InterpreterSelectQuery subquery. std::unique_ptr source; + QueryTreeNodePtr query_tree; }; class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this @@ -153,6 +157,8 @@ public: // void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } + SubqueryForSet & getSubquery() { return subquery; } + private: //SetPtr set; SubqueryForSet subquery; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 5f44994c14b..b3e3f5f472a 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -28,9 +28,9 @@ namespace class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor { public: - explicit CollectSetsVisitor(PlannerContext & planner_context_, const SelectQueryOptions & select_query_options_) + explicit CollectSetsVisitor(PlannerContext & planner_context_) //, const SelectQueryOptions & select_query_options_) : planner_context(planner_context_) - , select_query_options(select_query_options_) + //, select_query_options(select_query_options_) {} void visitImpl(const QueryTreeNodePtr & node) @@ -95,12 +95,12 @@ public: if (sets.getFuture(set_key)) return; - auto subquery_options = select_query_options.subquery(); - Planner subquery_planner( - in_second_argument, - subquery_options, - planner_context.getGlobalPlannerContext()); - subquery_planner.buildQueryPlanIfNeeded(); + // auto subquery_options = select_query_options.subquery(); + // Planner subquery_planner( + // in_second_argument, + // subquery_options, + // planner_context.getGlobalPlannerContext()); + // subquery_planner.buildQueryPlanIfNeeded(); // const auto & settings = planner_context.getQueryContext()->getSettingsRef(); // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; @@ -109,7 +109,8 @@ public: SubqueryForSet subquery_for_set; subquery_for_set.key = planner_context.createSetKey(in_second_argument); - subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + subquery_for_set.query_tree = in_second_argument; + //subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); /// TODO sets.addFromSubquery(set_key, std::move(subquery_for_set), settings, nullptr); @@ -132,14 +133,14 @@ public: private: PlannerContext & planner_context; - const SelectQueryOptions & select_query_options; + //const SelectQueryOptions & select_query_options; }; } -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options) +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) //, const SelectQueryOptions & select_query_options) { - CollectSetsVisitor visitor(planner_context, select_query_options); + CollectSetsVisitor visitor(planner_context); //, select_query_options); visitor.visit(node); } diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index 57e662a392e..e0db802d3b4 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -12,6 +12,6 @@ struct SelectQueryOptions; /** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations. * Collected sets are registered in planner context. */ -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context, const SelectQueryOptions & select_query_options); +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); //, const SelectQueryOptions & select_query_options); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 3184e229c15..30510d05840 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -893,50 +894,62 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana query_plan.addStep(std::move(offsets_step)); } -// void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan, -// const SelectQueryOptions & select_query_options, -// const PlannerContextPtr & planner_context, -// const std::vector & result_actions_to_execute) -// { -// PreparedSets::SubqueriesForSets subqueries_for_sets; +void addBuildSubqueriesForSetsStepIfNeeded( + QueryPlan & query_plan, + const SelectQueryOptions & select_query_options, + const PlannerContextPtr & planner_context, + const std::vector & result_actions_to_execute) +{ + auto subqueries = planner_context->getPreparedSets().detachSubqueries(); + std::unordered_set useful_sets; -// for (const auto & actions_to_execute : result_actions_to_execute) -// { -// for (const auto & node : actions_to_execute->getNodes()) -// { -// const auto & set_key = node.result_name; -// auto * planner_set = planner_context->getSetOrNull(set_key); -// if (!planner_set) -// continue; + PreparedSets::SubqueriesForSets subqueries_for_sets; -// if (planner_set->getSet().isCreated() || !planner_set->getSubqueryNode()) -// continue; + for (const auto & actions_to_execute : result_actions_to_execute) + { + for (const auto & node : actions_to_execute->getNodes()) + { + if (node.column) + { + const IColumn * column = node.column.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); -// auto subquery_options = select_query_options.subquery(); -// Planner subquery_planner( -// planner_set->getSubqueryNode(), -// subquery_options, -// planner_context->getGlobalPlannerContext()); -// subquery_planner.buildQueryPlanIfNeeded(); + if (const auto * column_set = typeid_cast(column)) + useful_sets.insert(column_set->getData().get()); + } + } + } -// const auto & settings = planner_context->getQueryContext()->getSettingsRef(); -// SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; -// bool tranform_null_in = settings.transform_null_in; -// auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); + auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.set.get()); }; + auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); + subqueries.erase(it, subqueries.end()); -// SubqueryForSet subquery_for_set; -// subquery_for_set.key = set_key; -// subquery_for_set.set_in_progress = set; -// subquery_for_set.set = planner_set->getSet(); -// subquery_for_set.promise_to_fill_set = planner_set->extractPromiseToBuildSet(); -// subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + for (auto & subquery : subqueries) + { + auto & subquery_for_set = subquery.set->getSubquery(); + auto subquery_options = select_query_options.subquery(); + Planner subquery_planner( + subquery_for_set.query_tree, + subquery_options, + planner_context->getGlobalPlannerContext()); + subquery_planner.buildQueryPlanIfNeeded(); -// subqueries_for_sets.emplace(set_key, std::move(subquery_for_set)); -// } -// } + subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + } -// addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); -// } + //addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); + + if (!subqueries.empty()) + { + auto step = std::make_unique( + query_plan.getCurrentDataStream(), + std::move(subqueries), + planner_context->getQueryContext()); + + query_plan.addStep(std::move(step)); + } +} /// Support for `additional_result_filter` setting void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, @@ -1197,7 +1210,7 @@ void Planner::buildPlanForQueryNode() } checkStoragesSupportTransactions(planner_context); - collectSets(query_tree, *planner_context, select_query_options); + collectSets(query_tree, *planner_context); //, select_query_options); collectTableExpressionData(query_tree, planner_context); const auto & settings = query_context->getSettingsRef(); @@ -1497,20 +1510,8 @@ void Planner::buildPlanForQueryNode() if (!select_query_options.only_analyze) { - auto subqueries = planner_context->getPreparedSets().detachSubqueries(); - - if (!subqueries.empty()) - { - auto step = std::make_unique( - query_plan.getCurrentDataStream(), - std::move(subqueries), - planner_context->getQueryContext()); - - query_plan.addStep(std::move(step)); - } - //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); - //addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); + addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); } } diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 2b4febf58ea..372bb15822a 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -426,7 +426,7 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options, + [[maybe_unused]] const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter) { const auto & query_context = planner_context->getQueryContext(); @@ -444,7 +444,7 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, } collectSourceColumns(filter_query_tree, planner_context); - collectSets(filter_query_tree, *planner_context, select_query_options); + collectSets(filter_query_tree, *planner_context); //, select_query_options); auto filter_actions_dag = std::make_shared(); diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index b0fb07d69ce..64a621e5710 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include @@ -898,9 +899,13 @@ QueryTreeNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, temporary_table_expression_node->setTemporaryTableName(temporary_table_name); auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context); - auto io = interpreter.execute(); - io.pipeline.complete(std::move(table_out)); - CompletedPipelineExecutor executor(io.pipeline); + + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(mutable_context); + auto build_pipeline_settings = BuildQueryPipelineSettings::fromContext(mutable_context); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline(optimization_settings, build_pipeline_settings))); + + pipeline.complete(std::move(table_out)); + CompletedPipelineExecutor executor(pipeline); executor.execute(); mutable_context->addExternalTable(temporary_table_name, std::move(external_storage_holder)); From e094bf3247b82c94098567a4f8f172e57fcdc017 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 9 Jun 2023 18:30:47 +0000 Subject: [PATCH 030/171] Resolving conflicts. --- src/Planner/CollectSets.cpp | 33 ++++++++++- .../QueryPlan/ReadFromMergeTree.cpp | 58 ++++++++++++++----- 2 files changed, 76 insertions(+), 15 deletions(-) diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index be4cb0e2e2b..68ad1ab78d3 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -96,6 +97,36 @@ public: if (sets.getFuture(set_key)) return; + auto subquery_to_execute = in_second_argument; + + if (auto * table_node = in_second_argument->as()) + { + auto storage_snapshot = table_node->getStorageSnapshot(); + auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); + + size_t columns_to_select_size = columns_to_select.size(); + + auto column_nodes_to_select = std::make_shared(); + column_nodes_to_select->getNodes().reserve(columns_to_select_size); + + NamesAndTypes projection_columns; + projection_columns.reserve(columns_to_select_size); + + for (auto & column : columns_to_select) + { + column_nodes_to_select->getNodes().emplace_back(std::make_shared(column, subquery_to_execute)); + projection_columns.emplace_back(column.name, column.type); + } + + auto subquery_for_table = std::make_shared(Context::createCopy(planner_context.getQueryContext())); + subquery_for_table->setIsSubquery(true); + subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select); + subquery_for_table->getJoinTree() = std::move(subquery_to_execute); + subquery_for_table->resolveProjectionColumns(std::move(projection_columns)); + + subquery_to_execute = std::move(subquery_for_table); + } + // auto subquery_options = select_query_options.subquery(); // Planner subquery_planner( // in_second_argument, @@ -110,7 +141,7 @@ public: SubqueryForSet subquery_for_set; subquery_for_set.key = planner_context.createSetKey(in_second_argument); - subquery_for_set.query_tree = in_second_argument; + subquery_for_set.query_tree = std::move(subquery_to_execute); //subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); /// TODO diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 8483df797ef..8f610eb4380 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -37,6 +37,8 @@ #include #include #include +#include +#include #include #include @@ -100,6 +102,7 @@ namespace ErrorCodes extern const int INDEX_NOT_USED; extern const int LOGICAL_ERROR; extern const int TOO_MANY_ROWS; + extern const int CANNOT_PARSE_TEXT; } static MergeTreeReaderSettings getMergeTreeReaderSettings( @@ -1245,29 +1248,56 @@ static void buildIndexes( info = &*info_copy; } + std::unordered_set ignored_index_names; + + if (settings.ignore_data_skipping_indices.changed) + { + const auto & indices = settings.ignore_data_skipping_indices.toString(); + Tokens tokens(indices.data(), indices.data() + indices.size(), settings.max_query_size); + IParser::Pos pos(tokens, static_cast(settings.max_parser_depth)); + Expected expected; + + /// Use an unordered list rather than string vector + auto parse_single_id_or_literal = [&] + { + String str; + if (!parseIdentifierOrStringLiteral(pos, expected, str)) + return false; + + ignored_index_names.insert(std::move(str)); + return true; + }; + + if (!ParserList::parseUtil(pos, expected, parse_single_id_or_literal, false)) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse ignore_data_skipping_indices ('{}')", indices); + } + UsefulSkipIndexes skip_indexes; using Key = std::pair; std::map merged; for (const auto & index : metadata_snapshot->getSecondaryIndices()) { - auto index_helper = MergeTreeIndexFactory::instance().get(index); - if (index_helper->isMergeable()) + if (!ignored_index_names.contains(index.name)) { - auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); - if (inserted) + auto index_helper = MergeTreeIndexFactory::instance().get(index); + if (index_helper->isMergeable()) { - skip_indexes.merged_indices.emplace_back(); - skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot); - } + auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size()); + if (inserted) + { + skip_indexes.merged_indices.emplace_back(); + skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot); + } - skip_indexes.merged_indices[it->second].addIndex(index_helper); - } - else - { - auto condition = index_helper->createIndexCondition(*info, context); - if (!condition->alwaysUnknownOrTrue()) - skip_indexes.useful_indices.emplace_back(index_helper, condition); + skip_indexes.merged_indices[it->second].addIndex(index_helper); + } + else + { + auto condition = index_helper->createIndexCondition(*info, context); + if (!condition->alwaysUnknownOrTrue()) + skip_indexes.useful_indices.emplace_back(index_helper, condition); + } } } From eb16745033fd34b5c9c32124b6bb870f7c795f9c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Jun 2023 11:04:07 +0000 Subject: [PATCH 031/171] Collect sets from indexHint actions as well. --- src/Planner/Planner.cpp | 44 ++++++++++++++++++++++++++++------------- 1 file changed, 30 insertions(+), 14 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index c1f472bb5a8..4ac81e28f92 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -895,6 +896,34 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana query_plan.addStep(std::move(offsets_step)); } +void collectSetsFromActionsDAG(const ActionsDAGPtr & dag, std::unordered_set & useful_sets) +{ + for (const auto & node : dag->getNodes()) + { + if (node.column) + { + const IColumn * column = node.column.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); + + if (const auto * column_set = typeid_cast(column)) + useful_sets.insert(column_set->getData().get()); + } + + if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint") + { + ActionsDAG::NodeRawConstPtrs children; + if (const auto * adaptor = typeid_cast(node.function_base.get())) + { + if (const auto * index_hint = typeid_cast(adaptor->getFunction().get())) + { + collectSetsFromActionsDAG(index_hint->getActions(), useful_sets); + } + } + } + } +} + void addBuildSubqueriesForSetsStepIfNeeded( QueryPlan & query_plan, const SelectQueryOptions & select_query_options, @@ -907,20 +936,7 @@ void addBuildSubqueriesForSetsStepIfNeeded( PreparedSets::SubqueriesForSets subqueries_for_sets; for (const auto & actions_to_execute : result_actions_to_execute) - { - for (const auto & node : actions_to_execute->getNodes()) - { - if (node.column) - { - const IColumn * column = node.column.get(); - if (const auto * column_const = typeid_cast(column)) - column = &column_const->getDataColumn(); - - if (const auto * column_set = typeid_cast(column)) - useful_sets.insert(column_set->getData().get()); - } - } - } + collectSetsFromActionsDAG(actions_to_execute, useful_sets); auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.set.get()); }; auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); From 70252321750b6a8d3ab6c41f658b76705a2e55b9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 12 Jun 2023 18:59:07 +0000 Subject: [PATCH 032/171] Fixing cache for sets. --- src/Processors/Transforms/CreatingSetsTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index a1e43525ab1..3139fa5ed98 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -86,7 +86,7 @@ void CreatingSetsTransform::startSubquery() } } - if (subquery.set) + if (subquery.set && !set_from_cache) LOG_TRACE(log, "Creating set, key: {}", subquery.key); if (subquery.table) LOG_TRACE(log, "Filling temporary table."); @@ -97,7 +97,7 @@ void CreatingSetsTransform::startSubquery() /// TODO: make via port table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), nullptr, /*async_insert=*/false)); - done_with_set = !subquery.set; + done_with_set = !subquery.set || set_from_cache; done_with_table = !subquery.table; if ((done_with_set && !set_from_cache) /*&& done_with_join*/ && done_with_table) @@ -175,10 +175,10 @@ void CreatingSetsTransform::consume(Chunk chunk) Chunk CreatingSetsTransform::generate() { - if (subquery.set) + if (subquery.set && !set_from_cache) { subquery.set->finishInsert(); - subquery.promise_to_fill_set.set_value(subquery.set); + //subquery.promise_to_fill_set.set_value(subquery.set); if (promise_to_build) promise_to_build->set_value(subquery.set); } From bf69755adab474fbd166209ab7675537d1a9aeeb Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 13 Jun 2023 01:18:36 -0700 Subject: [PATCH 033/171] Address some usability issues with INTO OUTFILE usage. --- src/Client/ClientBase.cpp | 20 ++++++++++++++++++++ src/Parsers/ASTQueryWithOutput.cpp | 2 ++ src/Parsers/ASTQueryWithOutput.h | 1 + src/Parsers/ParserQueryWithOutput.cpp | 6 ++++++ 4 files changed, 29 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6f295c11070..fc108b8f57d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -568,6 +568,13 @@ try CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); UInt64 compression_level = 3; + if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); + } + if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) { throw Exception( @@ -589,9 +596,22 @@ try range.second); } + std::error_code ec; + if (std::filesystem::is_regular_file(out_file, ec)) + { + if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::CANNOT_OPEN_FILE, + "File {} exists, consider using 'INTO OUTFILE ... APPEND' or 'INTO OUTFILE ... TRUNCATE' if appropriate.", + out_file); + } + } auto flags = O_WRONLY | O_EXCL; if (query_with_output->is_outfile_append) flags |= O_APPEND; + else if (query_with_output->is_outfile_truncate) + flags |= O_TRUNC; else flags |= O_CREAT; diff --git a/src/Parsers/ASTQueryWithOutput.cpp b/src/Parsers/ASTQueryWithOutput.cpp index 5f717715a69..4bf1e6cb231 100644 --- a/src/Parsers/ASTQueryWithOutput.cpp +++ b/src/Parsers/ASTQueryWithOutput.cpp @@ -39,6 +39,8 @@ void ASTQueryWithOutput::formatImpl(const FormatSettings & s, FormatState & stat s.ostr << (s.hilite ? hilite_keyword : ""); if (is_outfile_append) s.ostr << " APPEND"; + if (is_outfile_truncate) + s.ostr << " TRUNCATE"; if (is_into_outfile_with_stdout) s.ostr << " AND STDOUT"; s.ostr << (s.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTQueryWithOutput.h b/src/Parsers/ASTQueryWithOutput.h index 7db021405e7..6f9cafc89a9 100644 --- a/src/Parsers/ASTQueryWithOutput.h +++ b/src/Parsers/ASTQueryWithOutput.h @@ -17,6 +17,7 @@ public: ASTPtr out_file; bool is_into_outfile_with_stdout = false; bool is_outfile_append = false; + bool is_outfile_truncate = false; ASTPtr format; ASTPtr settings_ast; ASTPtr compression; diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 6796f4528c4..2bfe7353be4 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -109,6 +109,12 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query_with_output.is_outfile_append = true; } + ParserKeyword s_truncate("TRUNCATE"); + if (s_truncate.ignore(pos, expected)) + { + query_with_output.is_outfile_truncate = true; + } + ParserKeyword s_stdout("AND STDOUT"); if (s_stdout.ignore(pos, expected)) { From 7057e0e25fd55f8a9cb9708da223883aaa8fe902 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Jun 2023 14:46:26 +0000 Subject: [PATCH 034/171] fix test --- .../02751_ip_types_aggregate_functions_states.sql.j2 | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 index 708eeab7724..7daff5a690f 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -3,8 +3,8 @@ {# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} -{% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,50) order by ip" %} -{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 50) order by ip" %} +{% set ip4_generator = "select num::UInt32::IPv4 ip from (select arrayJoin(range(999999999, number)) as num from numbers(999999999,50)) order by ip" %} +{% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(num)||reinterpretAsFixedString(num), 16))) AS ip FROM (select arrayJoin(range(1010011101, number)) as num from numbers(1010011101,50)) order by ip" %} {% set ip_generators = {'ip4': ip4_generator, 'ip6': ip6_generator} %} From 332893344d3cbca205b0d99671cd4c8ba26ec2da Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 16:50:10 +0200 Subject: [PATCH 035/171] Updated lock for accessing azure blob storage iterator --- src/Storages/StorageAzureBlob.cpp | 6 +----- tests/integration/test_storage_azure_blob_storage/test.py | 1 - 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 3ee176a68b7..b9d59f04001 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -882,6 +882,7 @@ StorageAzureBlobSource::Iterator::Iterator( RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { + std::lock_guard lock(next_mutex); if (is_finished) return {}; @@ -900,7 +901,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { bool need_new_batch = false; { - std::lock_guard lock(next_mutex); need_new_batch = !blobs_with_metadata || index >= blobs_with_metadata->size(); } @@ -945,7 +945,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); const auto & idxs = typeid_cast(*block.getByName("_idx").column); - std::lock_guard lock(next_mutex); blob_path_with_globs.reset(); blob_path_with_globs.emplace(); for (UInt64 idx : idxs.getData()) @@ -961,7 +960,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() if (outer_blobs) outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); - std::lock_guard lock(next_mutex); blobs_with_metadata = std::move(new_batch); for (const auto & [_, info] : *blobs_with_metadata) total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); @@ -969,8 +967,6 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() } size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - - std::lock_guard lock(next_mutex); return (*blobs_with_metadata)[current_index]; } } diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f9d337b6d86..bb25ac4b029 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -551,7 +551,6 @@ def test_schema_inference_no_globs_tf(cluster): "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] - def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] unique_prefix = random.randint(1, 10000) From 3fd9911efe37500094532d07a1f6e0eaddaca6dd Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Jun 2023 14:58:55 +0000 Subject: [PATCH 036/171] fix test reference --- ...es_aggregate_functions_states.reference.j2 | 164 +++++++++--------- 1 file changed, 82 insertions(+), 82 deletions(-) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 index 481dd723b66..90f98cf63fd 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 @@ -2,18 +2,18 @@ Row 1: ────── minState: 12535288824949910799 -maxState: 18210943739258811465 +maxState: 15790547582231549775 first_valueState: 12535288824949910799 -last_valueState: 18210943739258811465 -topKState: 1594227852744382511 -groupArrayState: 8025417272361615478 -groupUniqArrayState: 919082878249747568 -uniqState: 14828781561416784358 -uniqExactState: 11391659146320471795 -uniqCombinedState: 9631896280254268221 -uniqCombined64State: 5156097536649078816 -uniqHLL12State: 9696624347265201099 -uniqThetaState: 10464560810701154023 +last_valueState: 15790547582231549775 +topKState: 4906125994014190470 +groupArrayState: 9875990674330641453 +groupUniqArrayState: 15788623094139372318 +uniqState: 13857072740564896081 +uniqExactState: 2764760571052675772 +uniqCombinedState: 927481020821507998 +uniqCombined64State: 11588178464185397904 +uniqHLL12State: 592773541433144605 +uniqThetaState: 12573391720108828030 ----- hash / State / ip6 ----- Row 1: ────── @@ -21,31 +21,31 @@ minState: 9428555662807296659 maxState: 18253481702148601156 first_valueState: 9428555662807296659 last_valueState: 18253481702148601156 -topKState: 8045294331733869941 -groupArrayState: 10451014709837753966 -groupUniqArrayState: 1954028114836070615 -uniqState: 14986562136250471284 -uniqExactState: 10032843621916709112 -uniqCombinedState: 6379274083567016598 -uniqCombined64State: 6379274083567016598 -uniqHLL12State: 9181286681186915812 -uniqThetaState: 2415188383468008881 +topKState: 4649040466816645853 +groupArrayState: 15334593495826890008 +groupUniqArrayState: 18179202420787216155 +uniqState: 1113473461736161202 +uniqExactState: 17291302630176581193 +uniqCombinedState: 7689865507370303115 +uniqCombined64State: 7689865507370303115 +uniqHLL12State: 12630756003012135681 +uniqThetaState: 11768246996604802350 ----- finalizeAggregation / State / ip4 ----- Row 1: ────── min: 59.154.201.255 -max: 59.154.202.48 +max: 59.154.202.47 first_value: 59.154.201.255 -last_value: 59.154.202.48 -topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] -groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] -groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 49 -uniqTheta: 50 +last_value: 59.154.202.47 +topK: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8'] +groupArray: ['59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.45','59.154.202.45','59.154.202.45','59.154.202.46','59.154.202.46','59.154.202.47'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 48 +uniqTheta: 49 ----- finalizeAggregation / State / ip6 ----- Row 1: ────── @@ -53,31 +53,31 @@ min: 8c:333c::8c:333c:0:0 max: ff8b:333c::ff8b:333c:0:0 first_value: 8c:333c::8c:333c:0:0 last_value: ff8b:333c::ff8b:333c:0:0 -topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] -groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 50 -uniqTheta: 50 +topK: ['dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 49 +uniqTheta: 49 ----- hash / IfState / ip4 ----- Row 1: ────── minIfState: 12535288824949910799 -maxIfState: 18210943739258811465 +maxIfState: 15790547582231549775 first_valueIfState: 12535288824949910799 -last_valueIfState: 18210943739258811465 -topKIfState: 1594227852744382511 -groupArrayIfState: 8025417272361615478 -groupUniqArrayIfState: 919082878249747568 -uniqIfState: 14828781561416784358 -uniqExactIfState: 11391659146320471795 -uniqCombinedIfState: 9631896280254268221 -uniqCombined64IfState: 5156097536649078816 -uniqHLL12IfState: 9696624347265201099 -uniqThetaIfState: 10464560810701154023 +last_valueIfState: 15790547582231549775 +topKIfState: 4906125994014190470 +groupArrayIfState: 9875990674330641453 +groupUniqArrayIfState: 15788623094139372318 +uniqIfState: 13857072740564896081 +uniqExactIfState: 2764760571052675772 +uniqCombinedIfState: 927481020821507998 +uniqCombined64IfState: 11588178464185397904 +uniqHLL12IfState: 592773541433144605 +uniqThetaIfState: 12573391720108828030 ----- hash / IfState / ip6 ----- Row 1: ────── @@ -85,31 +85,31 @@ minIfState: 9428555662807296659 maxIfState: 18253481702148601156 first_valueIfState: 9428555662807296659 last_valueIfState: 18253481702148601156 -topKIfState: 8045294331733869941 -groupArrayIfState: 10451014709837753966 -groupUniqArrayIfState: 1954028114836070615 -uniqIfState: 14986562136250471284 -uniqExactIfState: 10032843621916709112 -uniqCombinedIfState: 6379274083567016598 -uniqCombined64IfState: 6379274083567016598 -uniqHLL12IfState: 9181286681186915812 -uniqThetaIfState: 2415188383468008881 +topKIfState: 4649040466816645853 +groupArrayIfState: 15334593495826890008 +groupUniqArrayIfState: 18179202420787216155 +uniqIfState: 1113473461736161202 +uniqExactIfState: 17291302630176581193 +uniqCombinedIfState: 7689865507370303115 +uniqCombined64IfState: 7689865507370303115 +uniqHLL12IfState: 12630756003012135681 +uniqThetaIfState: 11768246996604802350 ----- finalizeAggregation / IfState / ip4 ----- Row 1: ────── min: 59.154.201.255 -max: 59.154.202.48 +max: 59.154.202.47 first_value: 59.154.201.255 -last_value: 59.154.202.48 -topK: ['59.154.202.48','59.154.202.5','59.154.202.26','59.154.202.25','59.154.202.24','59.154.202.23','59.154.202.22','59.154.202.21','59.154.202.27','59.154.202.19'] -groupArray: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8','59.154.202.9','59.154.202.10','59.154.202.11','59.154.202.12','59.154.202.13','59.154.202.14','59.154.202.15','59.154.202.16','59.154.202.17','59.154.202.18','59.154.202.19','59.154.202.20','59.154.202.21','59.154.202.22','59.154.202.23','59.154.202.24','59.154.202.25','59.154.202.26','59.154.202.27','59.154.202.28','59.154.202.29','59.154.202.30','59.154.202.31','59.154.202.32','59.154.202.33','59.154.202.34','59.154.202.35','59.154.202.36','59.154.202.37','59.154.202.38','59.154.202.39','59.154.202.40','59.154.202.41','59.154.202.42','59.154.202.43','59.154.202.44','59.154.202.45','59.154.202.46','59.154.202.47','59.154.202.48'] -groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.48','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 49 -uniqTheta: 50 +last_value: 59.154.202.47 +topK: ['59.154.201.255','59.154.202.0','59.154.202.1','59.154.202.2','59.154.202.3','59.154.202.4','59.154.202.5','59.154.202.6','59.154.202.7','59.154.202.8'] +groupArray: ['59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.201.255','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.0','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.1','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.2','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.3','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.4','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.5','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.6','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.7','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.8','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.9','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.10','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.11','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.12','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.13','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.14','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.15','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.16','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.17','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.18','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.19','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.20','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.21','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.22','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.23','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.24','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.25','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.26','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.27','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.28','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.29','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.30','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.31','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.32','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.33','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.34','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.35','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.36','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.37','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.38','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.39','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.40','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.41','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.42','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.43','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.44','59.154.202.45','59.154.202.45','59.154.202.45','59.154.202.46','59.154.202.46','59.154.202.47'] +groupUniqArray: ['59.154.202.28','59.154.202.45','59.154.202.35','59.154.202.2','59.154.202.42','59.154.202.1','59.154.202.4','59.154.202.15','59.154.202.22','59.154.202.20','59.154.202.12','59.154.202.3','59.154.202.40','59.154.202.43','59.154.202.26','59.154.202.37','59.154.202.7','59.154.202.36','59.154.202.32','59.154.202.47','59.154.202.17','59.154.202.11','59.154.201.255','59.154.202.0','59.154.202.14','59.154.202.25','59.154.202.6','59.154.202.30','59.154.202.16','59.154.202.21','59.154.202.23','59.154.202.38','59.154.202.44','59.154.202.39','59.154.202.41','59.154.202.27','59.154.202.33','59.154.202.19','59.154.202.5','59.154.202.9','59.154.202.18','59.154.202.24','59.154.202.34','59.154.202.46','59.154.202.8','59.154.202.29','59.154.202.10','59.154.202.13','59.154.202.31'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 48 +uniqTheta: 49 ----- finalizeAggregation / IfState / ip6 ----- Row 1: ────── @@ -117,20 +117,20 @@ min: 8c:333c::8c:333c:0:0 max: ff8b:333c::ff8b:333c:0:0 first_value: 8c:333c::8c:333c:0:0 last_value: ff8b:333c::ff8b:333c:0:0 -topK: ['ff8b:333c::ff8b:333c:0:0','68c:333c::68c:333c:0:0','e98b:333c::e98b:333c:0:0','e88b:333c::e88b:333c:0:0','e78b:333c::e78b:333c:0:0','e68b:333c::e68b:333c:0:0','e58b:333c::e58b:333c:0:0','e48b:333c::e48b:333c:0:0','ea8b:333c::ea8b:333c:0:0','e28b:333c::e28b:333c:0:0'] -groupArray: ['8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','e8c:333c::e8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e8c:333c::e8c:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] -uniq: 50 -uniqExact: 50 -uniqCombined: 50 -uniqCombined64: 50 -uniqHLL12: 50 -uniqTheta: 50 +topK: ['dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0'] +groupArray: ['8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','8c:333c::8c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','18c:333c::18c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','28c:333c::28c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','38c:333c::38c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','48c:333c::48c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','58c:333c::58c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','68c:333c::68c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','78c:333c::78c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','88c:333c::88c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','98c:333c::98c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','a8c:333c::a8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','b8c:333c::b8c:333c:0:0','c8c:333c::c8c:333c:0:0','c8c:333c::c8c:333c:0:0','d8c:333c::d8c:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','dd8b:333c::dd8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','de8b:333c::de8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','df8b:333c::df8b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e08b:333c::e08b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e18b:333c::e18b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e28b:333c::e28b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e48b:333c::e48b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e58b:333c::e58b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e68b:333c::e68b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e78b:333c::e78b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e88b:333c::e88b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','e98b:333c::e98b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ec8b:333c::ec8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ed8b:333c::ed8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f08b:333c::f08b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f18b:333c::f18b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f28b:333c::f28b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f38b:333c::f38b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f48b:333c::f48b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f58b:333c::f58b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f68b:333c::f68b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f78b:333c::f78b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f88b:333c::f88b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','f98b:333c::f98b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fa8b:333c::fa8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fc8b:333c::fc8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fd8b:333c::fd8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','fe8b:333c::fe8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +groupUniqArray: ['58c:333c::58c:333c:0:0','f78b:333c::f78b:333c:0:0','f38b:333c::f38b:333c:0:0','18c:333c::18c:333c:0:0','e78b:333c::e78b:333c:0:0','e38b:333c::e38b:333c:0:0','e48b:333c::e48b:333c:0:0','e08b:333c::e08b:333c:0:0','df8b:333c::df8b:333c:0:0','f48b:333c::f48b:333c:0:0','68c:333c::68c:333c:0:0','28c:333c::28c:333c:0:0','f08b:333c::f08b:333c:0:0','fa8b:333c::fa8b:333c:0:0','88c:333c::88c:333c:0:0','c8c:333c::c8c:333c:0:0','fe8b:333c::fe8b:333c:0:0','ea8b:333c::ea8b:333c:0:0','ee8b:333c::ee8b:333c:0:0','e98b:333c::e98b:333c:0:0','ed8b:333c::ed8b:333c:0:0','b8c:333c::b8c:333c:0:0','f98b:333c::f98b:333c:0:0','fd8b:333c::fd8b:333c:0:0','de8b:333c::de8b:333c:0:0','f58b:333c::f58b:333c:0:0','78c:333c::78c:333c:0:0','38c:333c::38c:333c:0:0','f18b:333c::f18b:333c:0:0','e58b:333c::e58b:333c:0:0','e18b:333c::e18b:333c:0:0','e68b:333c::e68b:333c:0:0','e28b:333c::e28b:333c:0:0','48c:333c::48c:333c:0:0','dd8b:333c::dd8b:333c:0:0','f68b:333c::f68b:333c:0:0','f28b:333c::f28b:333c:0:0','8c:333c::8c:333c:0:0','a8c:333c::a8c:333c:0:0','f88b:333c::f88b:333c:0:0','fc8b:333c::fc8b:333c:0:0','e88b:333c::e88b:333c:0:0','ec8b:333c::ec8b:333c:0:0','eb8b:333c::eb8b:333c:0:0','ef8b:333c::ef8b:333c:0:0','fb8b:333c::fb8b:333c:0:0','98c:333c::98c:333c:0:0','d8c:333c::d8c:333c:0:0','ff8b:333c::ff8b:333c:0:0'] +uniq: 49 +uniqExact: 49 +uniqCombined: 49 +uniqCombined64: 49 +uniqHLL12: 49 +uniqTheta: 49 ----- Arg / hash / State / ip4 ----- Row 1: ────── argMinState: 13774589216353164344 -argMaxState: 9177365218111013695 +argMaxState: 1131101663917664667 ----- Arg / hash / State / ip6 ----- Row 1: ────── @@ -140,7 +140,7 @@ argMaxState: 16598449636475438091 Row 1: ────── argMinState: 59.154.201.255 -argMaxState: 59.154.202.48 +argMaxState: 59.154.202.47 ----- Arg / finalizeAggregation / State / ip6 ----- Row 1: ────── From 478bad32376ff2787e83107c8274f4b743c569ac Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Jun 2023 15:04:11 +0000 Subject: [PATCH 037/171] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index bb25ac4b029..f9d337b6d86 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -551,6 +551,7 @@ def test_schema_inference_no_globs_tf(cluster): "499500\t2890\t332833500\ttest_schema_inference_no_globs_tf.csv\tcont/test_schema_inference_no_globs_tf.csv" ] + def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] unique_prefix = random.randint(1, 10000) From bc4724490239ea34b4924da17ddabbb1f90e2bee Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 13 Jun 2023 17:06:40 +0200 Subject: [PATCH 038/171] Updated tests for CI checks --- tests/integration/test_storage_azure_blob_storage/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index f9d337b6d86..8ab5d416b03 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -558,6 +558,7 @@ def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" + for i in range(10): for j in range(10): path = "{}/{}_{}/{}.csv".format( From 8dde50eb3aeef2bd83a1c7c0ec244dfe1c66bf32 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 13 Jun 2023 15:19:16 +0000 Subject: [PATCH 039/171] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 8ab5d416b03..3d9c751be3c 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -558,7 +558,7 @@ def test_schema_inference_from_globs_tf(cluster): node = cluster.instances["node"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" - + for i in range(10): for j in range(10): path = "{}/{}_{}/{}.csv".format( From e7d2cc4a1a44e9c52d16ede99612e28342735c15 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 13 Jun 2023 22:11:23 +0000 Subject: [PATCH 040/171] add map functions tests --- ...es_aggregate_functions_states.reference.j2 | 48 ++++++++++++++ ...ip_types_aggregate_functions_states.sql.j2 | 64 ++++++++++++++++--- 2 files changed, 102 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 index 90f98cf63fd..03b0e065151 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.reference.j2 @@ -170,3 +170,51 @@ Row 1: any: dd8b:333c::dd8b:333c:0:0 anyHeavy: dd8b:333c::dd8b:333c:0:0 anyLast: dd8b:333c::dd8b:333c:0:0 +----- Map/Map hash / State / ip4 ----- +Row 1: +────── +sumMapState: 9327034461443333306 +minMapState: 17403430892851901033 +maxMapState: 17403430892851901033 +----- Map/Map hash / State / ip6 ----- +Row 1: +────── +sumMapState: 5204122423200337352 +minMapState: 14534921151579960284 +maxMapState: 14534921151579960284 +----- Map/Map finalizeAggregation / State / ip4 ----- +Row 1: +────── +sumMap: {'59.154.201.255':1} +minMap: {'59.154.201.255':1} +maxMap: {'59.154.201.255':1} +----- Map/Map finalizeAggregation / State / ip6 ----- +Row 1: +────── +sumMap: {'dd8b:333c::dd8b:333c:0:0':1} +minMap: {'dd8b:333c::dd8b:333c:0:0':1} +maxMap: {'dd8b:333c::dd8b:333c:0:0':1} +----- Map/Array hash / State / ip4 ----- +Row 1: +────── +sumMapState: 9327034461443333306 +minMapState: 9327034461443333306 +maxMapState: 9327034461443333306 +----- Map/Array hash / State / ip6 ----- +Row 1: +────── +sumMapState: 15555709096566410627 +minMapState: 15555709096566410627 +maxMapState: 15555709096566410627 +----- Map/Array finalizeAggregation / State / ip4 ----- +Row 1: +────── +sumMap: (['59.154.201.255'],[1]) +minMap: (['59.154.201.255'],[1]) +maxMap: (['59.154.201.255'],[1]) +----- Map/Array finalizeAggregation / State / ip6 ----- +Row 1: +────── +sumMap: (['dd8b:333c::dd8b:333c:0:0'],[1]) +minMap: (['dd8b:333c::dd8b:333c:0:0'],[1]) +maxMap: (['dd8b:333c::dd8b:333c:0:0'],[1]) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 index 7daff5a690f..7d030d4be2d 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -1,5 +1,4 @@ --- Tags: no-fasttest --- no-fasttest because uniqTheta +-- Tags: no-parallel, no-fasttest {# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} @@ -22,7 +21,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- finalizeAggregation / State / {{ generator_name }} -----'; @@ -35,7 +33,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- hash / IfState / {{ generator_name }} -----'; @@ -48,7 +45,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- finalizeAggregation / IfState / {{ generator_name }} -----'; @@ -61,8 +57,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - - {% set agg_func_list = [ "argMin", "argMax" ] %} {% for generator_name, ip_generator in ip_generators.items() %} @@ -77,8 +71,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - - {% for generator_name, ip_generator in ip_generators.items() %} select '----- Arg / finalizeAggregation / State / {{ generator_name }} -----'; @@ -91,6 +83,8 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} + + {# let's test functions with not deterministic result against 1 row, to make it deterministic #} {% set ip4_generator = "select number::UInt32::IPv4 ip from numbers(999999999,1) order by ip" %} {% set ip6_generator = "SELECT toIPv6(IPv6NumToString(toFixedString(reinterpretAsFixedString(number)||reinterpretAsFixedString(number), 16))) AS ip FROM numbers(1010011101, 1) order by ip" %} @@ -112,7 +106,6 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} - {% for generator_name, ip_generator in ip_generators.items() %} select '----- finalizeAggregation / State / {{ generator_name }} -----'; @@ -124,3 +117,54 @@ from ( {{ ip_generator }} ) format Vertical; {% endfor -%} + + +{% set agg_func_list = [ "sumMap", "minMap", "maxMap" ] %} + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Map hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State(map(ip, 1::Int64)) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Map finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State(map(ip, 1::Int64)) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Array hash / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + cityHash64(hex( {{ func }}State([ip], [1::Int64]) )) AS {{ func }}State{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} + + +{% for generator_name, ip_generator in ip_generators.items() %} + +select '----- Map/Array finalizeAggregation / State / {{ generator_name }} -----'; +select + {% for func in agg_func_list -%} + finalizeAggregation( {{ func }}State([ip], [1::Int64]) ) AS {{ func }}{{ "," if not loop.last }} + {% endfor -%} +from ( {{ ip_generator }} ) format Vertical; + +{% endfor -%} From 0a1d0c4abd0f8ece5af6f3a3d5ccc5207dfff0f2 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 13 Jun 2023 23:36:14 -0700 Subject: [PATCH 041/171] Fix one stateless test. --- src/Client/ClientBase.cpp | 5 ++--- tests/queries/0_stateless/02346_into_outfile_and_stdout.sh | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index fc108b8f57d..b746d46148c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -596,14 +596,13 @@ try range.second); } - std::error_code ec; - if (std::filesystem::is_regular_file(out_file, ec)) + if (fs::exists(out_file)) { if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) { throw Exception( ErrorCodes::CANNOT_OPEN_FILE, - "File {} exists, consider using 'INTO OUTFILE ... APPEND' or 'INTO OUTFILE ... TRUNCATE' if appropriate.", + "File {} exists, consider using APPEND or TRUNCATE if appropriate.", out_file); } } diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh index 021dc9125d4..8ec086c97ef 100755 --- a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh @@ -66,7 +66,7 @@ performBadQuery "bad_query_incorrect_usage" "SELECT 1, 2, 3 INTO OUTFILE AND STD performBadQuery "bad_query_no_into_outfile" "SELECT 1, 2, 3 AND STDOUT'" "SYNTAX_ERROR" -performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File exists. (CANNOT_OPEN_FILE)" +performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File ${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists exists, consider using APPEND or TRUNCATE if appropriate. (CANNOT_OPEN_FILE)" performCompression "compression" "SELECT * FROM (SELECT 'Hello, World! From clickhouse.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_compression.gz' AND STDOUT COMPRESSION 'GZ' FORMAT TabSeparated" From 1663905acdafcce577d58852a0835fdb760750e8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 14 Jun 2023 08:48:16 +0000 Subject: [PATCH 042/171] Hold context in pipeline resources. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 1 + src/QueryPipeline/QueryPipelineBuilder.h | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 8f610eb4380..bbea1d38fb1 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1884,6 +1884,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons processors.emplace_back(processor); pipeline.init(std::move(pipe)); + pipeline.addContext(context); // Attach QueryIdHolder if needed if (query_id_holder) pipeline.setQueryIdHolder(std::move(query_id_holder)); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index da8443a7e33..134de59520d 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -185,6 +185,7 @@ public: void addResources(QueryPlanResourceHolder resources_) { resources = std::move(resources_); } void setQueryIdHolder(std::shared_ptr query_id_holder) { resources.query_id_holders.emplace_back(std::move(query_id_holder)); } + void addContext(ContextPtr context) { resources.interpreter_context.emplace_back(std::move(context)); } /// Convert query pipeline to pipe. static Pipe getPipe(QueryPipelineBuilder pipeline, QueryPlanResourceHolder & resources); From 918b8c4585025e8a357115945cce501c05d31be1 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 14 Jun 2023 10:51:59 +0200 Subject: [PATCH 043/171] Updated filename in test --- tests/integration/test_storage_azure_blob_storage/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 3d9c751be3c..e99ae72eb8b 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -583,13 +583,13 @@ def test_partition_by_tf(cluster): table_format = "column1 UInt32, column2 UInt32, column3 UInt32" partition_by = "column3" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" - filename = "test_tf_{_partition_id}.csv" + filename = "test_partition_tf_{_partition_id}.csv" azure_query( node, f"INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", ) - assert "1,2,3\n" == get_azure_file_content("test_tf_3.csv") - assert "3,2,1\n" == get_azure_file_content("test_tf_1.csv") - assert "78,43,45\n" == get_azure_file_content("test_tf_45.csv") + assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv") + assert "3,2,1\n" == get_azure_file_content("test_partition_tf_1.csv") + assert "78,43,45\n" == get_azure_file_content("test_partition_tfs_45.csv") From c98a194b571e8c39504afc829fa91492f4dcbe2d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 14 Jun 2023 11:00:11 +0200 Subject: [PATCH 044/171] Updated unique names for test to avoid same names by random numbers --- tests/integration/test_storage_azure_blob_storage/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index e99ae72eb8b..e2077f8face 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -300,10 +300,10 @@ def test_put_get_with_globs(cluster): azure_query( node, - f"CREATE TABLE test_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + f"CREATE TABLE test_put_{i}_{j} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", ) - query = f"insert into test_{i}_{j} VALUES {values}" + query = f"insert into test_put_{i}_{j} VALUES {values}" azure_query(node, query) azure_query( @@ -332,9 +332,9 @@ def test_azure_glob_scheherazade(cluster): unique_num = random.randint(1, 10000) azure_query( node, - f"CREATE TABLE test_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", + f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", ) - query = f"insert into test_{i}_{unique_num} VALUES {values}" + query = f"insert into test_scheherazade_{i}_{unique_num} VALUES {values}" azure_query(node, query) jobs.append( From f6bad2c064efeb997755be8b9f313a3859a6f81f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 14 Jun 2023 09:13:38 +0000 Subject: [PATCH 045/171] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index e2077f8face..0002ccbf483 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -334,7 +334,9 @@ def test_azure_glob_scheherazade(cluster): node, f"CREATE TABLE test_scheherazade_{i}_{unique_num} ({table_format}) Engine = AzureBlobStorage(azure_conf2, container='cont', blob_path='{path}', format='CSV')", ) - query = f"insert into test_scheherazade_{i}_{unique_num} VALUES {values}" + query = ( + f"insert into test_scheherazade_{i}_{unique_num} VALUES {values}" + ) azure_query(node, query) jobs.append( From 011d666073968b1a8cbbd867513e4e8adec1362b Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 14 Jun 2023 14:55:34 +0200 Subject: [PATCH 046/171] Fixed typo in tests --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 0002ccbf483..0de325ccd14 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -594,4 +594,4 @@ def test_partition_by_tf(cluster): assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv") assert "3,2,1\n" == get_azure_file_content("test_partition_tf_1.csv") - assert "78,43,45\n" == get_azure_file_content("test_partition_tfs_45.csv") + assert "78,43,45\n" == get_azure_file_content("test_partition_tf_45.csv") From 7f8162e3465318bb9847956edffcd67a4fdb2020 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 14 Jun 2023 10:42:20 -0400 Subject: [PATCH 047/171] add note regarding -MapState --- docs/en/sql-reference/aggregate-functions/combinators.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/combinators.md b/docs/en/sql-reference/aggregate-functions/combinators.md index fd693430064..a395b350a55 100644 --- a/docs/en/sql-reference/aggregate-functions/combinators.md +++ b/docs/en/sql-reference/aggregate-functions/combinators.md @@ -97,6 +97,10 @@ Result: If you apply this combinator, the aggregate function does not return the resulting value (such as the number of unique values for the [uniq](../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function), but an intermediate state of the aggregation (for `uniq`, this is the hash table for calculating the number of unique values). This is an `AggregateFunction(...)` that can be used for further processing or stored in a table to finish aggregating later. +:::note +Please notice, that -MapState is not an invariant for the same data due to the fact that order of data in intermediate state can change, though it doesn't impact ingestion of this data. +::: + To work with these states, use: - [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md) table engine. From 0131ca8768efd5840f330b2ef11acc72c35a36a2 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 14 Jun 2023 10:49:20 -0400 Subject: [PATCH 048/171] add note regarding -MapState --- docs/ru/sql-reference/aggregate-functions/combinators.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/ru/sql-reference/aggregate-functions/combinators.md b/docs/ru/sql-reference/aggregate-functions/combinators.md index 3a7ff571f99..99d5f11442c 100644 --- a/docs/ru/sql-reference/aggregate-functions/combinators.md +++ b/docs/ru/sql-reference/aggregate-functions/combinators.md @@ -66,6 +66,10 @@ WITH anySimpleState(number) AS c SELECT toTypeName(c), c FROM numbers(1); В случае применения этого комбинатора, агрегатная функция возвращает не готовое значение (например, в случае функции [uniq](reference/uniq.md#agg_function-uniq) — количество уникальных значений), а промежуточное состояние агрегации (например, в случае функции `uniq` — хэш-таблицу для расчёта количества уникальных значений), которое имеет тип `AggregateFunction(...)` и может использоваться для дальнейшей обработки или может быть сохранено в таблицу для последующей доагрегации. +:::note +Промежуточное состояние для -MapState не является инвариантом для одних и тех же исходных данные т.к. порядок данных может меняться. Это не влияет, тем не менее, на загрузку таких данных. +::: + Для работы с промежуточными состояниями предназначены: - Движок таблиц [AggregatingMergeTree](../../engines/table-engines/mergetree-family/aggregatingmergetree.md). From d0d39c8f6142cf2b954b3125eca4491f82718524 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 14 Jun 2023 18:10:30 +0000 Subject: [PATCH 049/171] Trying to fix a test. --- src/Interpreters/ActionsVisitor.cpp | 29 ++++++- .../InterpreterSelectQueryAnalyzer.h | 2 + src/Interpreters/interpretSubquery.cpp | 85 ++++++++++++------- src/Interpreters/interpretSubquery.h | 5 ++ src/Planner/CollectSets.cpp | 1 + 5 files changed, 87 insertions(+), 35 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 59bbc74ca3a..34c3dab8926 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -3,7 +3,8 @@ #include #include #include -#include "Parsers/queryToString.h" +#include +#include #include #include @@ -55,6 +56,7 @@ #include #include #include +#include #include #include @@ -1394,7 +1396,18 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (no_subqueries) return {}; //std::cerr << queryToString(right_in_operand) << std::endl; - auto set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); + PreparedSetKey set_key; + if (data.getContext()->getSettingsRef().allow_experimental_analyzer) + { + InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery()); + auto query_tree = interpreter.getQueryTree(); + if (auto * query_node = query_tree->as()) + query_node->setIsSubquery(true); + // std::cerr << "============== " << interpreter.getQueryTree()->dumpTree() << std::endl; + set_key = PreparedSetKey::forSubquery(interpreter.getQueryTree()->getTreeHash()); + } + else + set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); // std::cerr << set_key.toString() << std::endl; // std::cerr << data.prepared_sets->getSets().size() << std::endl; @@ -1446,8 +1459,16 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool * Also it doesn't make sense if it is GLOBAL IN or ordinary IN. */ { - auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); - subquery_for_set.createSource(*interpreter); + if (data.getContext()->getSettingsRef().allow_experimental_analyzer) + { + auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth); + subquery_for_set.source = std::make_unique(std::move(*interpreter).extractQueryPlan()); + } + else + { + auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); + subquery_for_set.createSource(*interpreter); + } } return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), data.getContext()->getSettingsRef(), std::move(external_table_set)); diff --git a/src/Interpreters/InterpreterSelectQueryAnalyzer.h b/src/Interpreters/InterpreterSelectQueryAnalyzer.h index 1e0ac737536..4434fabe746 100644 --- a/src/Interpreters/InterpreterSelectQueryAnalyzer.h +++ b/src/Interpreters/InterpreterSelectQueryAnalyzer.h @@ -69,6 +69,8 @@ public: const Planner & getPlanner() const { return planner; } Planner & getPlanner() { return planner; } + const QueryTreeNodePtr & getQueryTree() const { return query_tree; } + private: ASTPtr query; ContextMutablePtr context; diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index 5f00be07fa5..b757127a14d 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -21,37 +22,10 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns) +static ASTPtr buildQueryAST(const ASTPtr & table_expression, const ContextPtr & context, SelectQueryOptions & subquery_options); + +static ContextPtr getSubqueryContext(const ContextPtr & context) { - auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth); - return interpretSubquery(table_expression, context, required_source_columns, subquery_options); -} - -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, const Names & required_source_columns, const SelectQueryOptions & options) -{ - if (auto * expr = table_expression->as()) - { - ASTPtr table; - if (expr->subquery) - table = expr->subquery; - else if (expr->table_function) - table = expr->table_function; - else if (expr->database_and_table_name) - table = expr->database_and_table_name; - - return interpretSubquery(table, context, required_source_columns, options); - } - - /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. - const auto * subquery = table_expression->as(); - const auto * function = table_expression->as(); - const auto * table = table_expression->as(); - - if (!subquery && !table && !function) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery."); - /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. * Because the result of this query is not the result of the entire query. * Constraints work instead @@ -67,7 +41,56 @@ std::shared_ptr interpretSubquery( subquery_settings.extremes = false; subquery_context->setSettings(subquery_settings); + return subquery_context; +} + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns) +{ + auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth); + return interpretSubquery(table_expression, context, required_source_columns, subquery_options); +} + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, const Names & required_source_columns, const SelectQueryOptions & options) +{ auto subquery_options = options.subquery(); + auto query = buildQueryAST(table_expression, context, subquery_options); + auto subquery_context = getSubqueryContext(context); + return std::make_shared(query, subquery_context, subquery_options, required_source_columns); +} + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth) +{ + auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth).subquery(); + auto query = buildQueryAST(table_expression, context, subquery_options); + auto subquery_context = getSubqueryContext(context); + return std::make_shared(query, subquery_context, subquery_options); +} + +static ASTPtr buildQueryAST(const ASTPtr & table_expression, const ContextPtr & context, SelectQueryOptions & subquery_options) +{ + if (auto * expr = table_expression->as()) + { + ASTPtr table; + if (expr->subquery) + table = expr->subquery; + else if (expr->table_function) + table = expr->table_function; + else if (expr->database_and_table_name) + table = expr->database_and_table_name; + + return buildQueryAST(table, context, subquery_options); + } + + /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. + const auto * subquery = table_expression->as(); + const auto * function = table_expression->as(); + const auto * table = table_expression->as(); + + if (!subquery && !table && !function) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery."); ASTPtr query; if (table || function) @@ -112,7 +135,7 @@ std::shared_ptr interpretSubquery( subquery_options.removeDuplicates(); } - return std::make_shared(query, subquery_context, subquery_options, required_source_columns); + return query; } } diff --git a/src/Interpreters/interpretSubquery.h b/src/Interpreters/interpretSubquery.h index 3836d1f7664..02d2003ea5f 100644 --- a/src/Interpreters/interpretSubquery.h +++ b/src/Interpreters/interpretSubquery.h @@ -6,6 +6,11 @@ namespace DB { +class InterpreterSelectQueryAnalyzer; + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth); + std::shared_ptr interpretSubquery( const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns); diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 68ad1ab78d3..90f3f074761 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -93,6 +93,7 @@ public: in_second_argument_node_type == QueryTreeNodeType::UNION || in_second_argument_node_type == QueryTreeNodeType::TABLE) { + // std::cerr << "======2======= " << in_second_argument->dumpTree() << std::endl; auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); if (sets.getFuture(set_key)) return; From 35b7474ccfe009c0a2612f55bbca436eba9218a7 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 15 Jun 2023 10:22:58 -0400 Subject: [PATCH 050/171] add MapState to aspell-dict.txt --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index d6cef1883f4..79b1c2ae4db 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -468,6 +468,7 @@ MSan MVCC MacBook MacOS +MapState MarkCacheBytes MarkCacheFiles MarksLoaderThreads From d34b6e5f35280b916627359d01cbf7a4d07f447b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 15 Jun 2023 16:27:51 +0000 Subject: [PATCH 051/171] Try to fix analyzer tests. --- src/Interpreters/ActionsVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 34c3dab8926..393a25463ca 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1397,7 +1397,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return {}; //std::cerr << queryToString(right_in_operand) << std::endl; PreparedSetKey set_key; - if (data.getContext()->getSettingsRef().allow_experimental_analyzer) + if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery()); auto query_tree = interpreter.getQueryTree(); From 1cb7ba4895e0666eeba8654a45ec8c124b904864 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 15 Jun 2023 16:42:38 +0000 Subject: [PATCH 052/171] Revert "Trying to fix a test." This reverts commit d0d39c8f6142cf2b954b3125eca4491f82718524. --- src/Interpreters/ActionsVisitor.cpp | 21 +++---- src/Interpreters/interpretSubquery.cpp | 85 ++++++++++---------------- src/Interpreters/interpretSubquery.h | 5 -- src/Planner/CollectSets.cpp | 1 - 4 files changed, 38 insertions(+), 74 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 393a25463ca..4b38f8c9434 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -3,8 +3,6 @@ #include #include #include -#include -#include #include #include @@ -56,10 +54,12 @@ #include #include #include -#include #include #include +#include +#include +#include namespace DB { @@ -1396,6 +1396,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (no_subqueries) return {}; //std::cerr << queryToString(right_in_operand) << std::endl; + PreparedSetKey set_key; if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { @@ -1404,7 +1405,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (auto * query_node = query_tree->as()) query_node->setIsSubquery(true); // std::cerr << "============== " << interpreter.getQueryTree()->dumpTree() << std::endl; - set_key = PreparedSetKey::forSubquery(interpreter.getQueryTree()->getTreeHash()); + set_key = PreparedSetKey::forSubquery(query_tree->getTreeHash()); } else set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); @@ -1459,16 +1460,8 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool * Also it doesn't make sense if it is GLOBAL IN or ordinary IN. */ { - if (data.getContext()->getSettingsRef().allow_experimental_analyzer) - { - auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth); - subquery_for_set.source = std::make_unique(std::move(*interpreter).extractQueryPlan()); - } - else - { - auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); - subquery_for_set.createSource(*interpreter); - } + auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); + subquery_for_set.createSource(*interpreter); } return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), data.getContext()->getSettingsRef(), std::move(external_table_set)); diff --git a/src/Interpreters/interpretSubquery.cpp b/src/Interpreters/interpretSubquery.cpp index b757127a14d..5f00be07fa5 100644 --- a/src/Interpreters/interpretSubquery.cpp +++ b/src/Interpreters/interpretSubquery.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -22,10 +21,37 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static ASTPtr buildQueryAST(const ASTPtr & table_expression, const ContextPtr & context, SelectQueryOptions & subquery_options); - -static ContextPtr getSubqueryContext(const ContextPtr & context) +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns) { + auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth); + return interpretSubquery(table_expression, context, required_source_columns, subquery_options); +} + +std::shared_ptr interpretSubquery( + const ASTPtr & table_expression, ContextPtr context, const Names & required_source_columns, const SelectQueryOptions & options) +{ + if (auto * expr = table_expression->as()) + { + ASTPtr table; + if (expr->subquery) + table = expr->subquery; + else if (expr->table_function) + table = expr->table_function; + else if (expr->database_and_table_name) + table = expr->database_and_table_name; + + return interpretSubquery(table, context, required_source_columns, options); + } + + /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. + const auto * subquery = table_expression->as(); + const auto * function = table_expression->as(); + const auto * table = table_expression->as(); + + if (!subquery && !table && !function) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery."); + /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. * Because the result of this query is not the result of the entire query. * Constraints work instead @@ -41,56 +67,7 @@ static ContextPtr getSubqueryContext(const ContextPtr & context) subquery_settings.extremes = false; subquery_context->setSettings(subquery_settings); - return subquery_context; -} - -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns) -{ - auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth); - return interpretSubquery(table_expression, context, required_source_columns, subquery_options); -} - -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, const Names & required_source_columns, const SelectQueryOptions & options) -{ auto subquery_options = options.subquery(); - auto query = buildQueryAST(table_expression, context, subquery_options); - auto subquery_context = getSubqueryContext(context); - return std::make_shared(query, subquery_context, subquery_options, required_source_columns); -} - -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth) -{ - auto subquery_options = SelectQueryOptions(QueryProcessingStage::Complete, subquery_depth).subquery(); - auto query = buildQueryAST(table_expression, context, subquery_options); - auto subquery_context = getSubqueryContext(context); - return std::make_shared(query, subquery_context, subquery_options); -} - -static ASTPtr buildQueryAST(const ASTPtr & table_expression, const ContextPtr & context, SelectQueryOptions & subquery_options) -{ - if (auto * expr = table_expression->as()) - { - ASTPtr table; - if (expr->subquery) - table = expr->subquery; - else if (expr->table_function) - table = expr->table_function; - else if (expr->database_and_table_name) - table = expr->database_and_table_name; - - return buildQueryAST(table, context, subquery_options); - } - - /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. - const auto * subquery = table_expression->as(); - const auto * function = table_expression->as(); - const auto * table = table_expression->as(); - - if (!subquery && !table && !function) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table expression is undefined, Method: ExpressionAnalyzer::interpretSubquery."); ASTPtr query; if (table || function) @@ -135,7 +112,7 @@ static ASTPtr buildQueryAST(const ASTPtr & table_expression, const ContextPtr & subquery_options.removeDuplicates(); } - return query; + return std::make_shared(query, subquery_context, subquery_options, required_source_columns); } } diff --git a/src/Interpreters/interpretSubquery.h b/src/Interpreters/interpretSubquery.h index 02d2003ea5f..3836d1f7664 100644 --- a/src/Interpreters/interpretSubquery.h +++ b/src/Interpreters/interpretSubquery.h @@ -6,11 +6,6 @@ namespace DB { -class InterpreterSelectQueryAnalyzer; - -std::shared_ptr interpretSubquery( - const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth); - std::shared_ptr interpretSubquery( const ASTPtr & table_expression, ContextPtr context, size_t subquery_depth, const Names & required_source_columns); diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 90f3f074761..68ad1ab78d3 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -93,7 +93,6 @@ public: in_second_argument_node_type == QueryTreeNodeType::UNION || in_second_argument_node_type == QueryTreeNodeType::TABLE) { - // std::cerr << "======2======= " << in_second_argument->dumpTree() << std::endl; auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); if (sets.getFuture(set_key)) return; From f610645c077b9f96257a8d9ce7ced71d3ac6cbba Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Fri, 16 Jun 2023 01:10:06 -0700 Subject: [PATCH 053/171] Move outfile checks before query starts running. --- src/Client/ClientBase.cpp | 103 ++++++++++++++++++++++++++------------ 1 file changed, 70 insertions(+), 33 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b746d46148c..08e14f83982 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -103,6 +103,7 @@ namespace ErrorCodes extern const int UNRECOGNIZED_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int CANNOT_OPEN_FILE; + extern const int FILE_ALREADY_EXISTS; } } @@ -568,44 +569,12 @@ try CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); UInt64 compression_level = 3; - if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); - } - - if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot append to compressed file. Please use uncompressed file or remove APPEND keyword."); - } - if (query_with_output->compression_level) { const auto & compression_level_node = query_with_output->compression_level->as(); - bool res = compression_level_node.value.tryGet(compression_level); - auto range = getCompressionLevelRange(compression_method); - - if (!res || compression_level < range.first || compression_level > range.second) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Invalid compression level, must be positive integer in range {}-{}", - range.first, - range.second); + compression_level_node.value.tryGet(compression_level); } - if (fs::exists(out_file)) - { - if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) - { - throw Exception( - ErrorCodes::CANNOT_OPEN_FILE, - "File {} exists, consider using APPEND or TRUNCATE if appropriate.", - out_file); - } - } auto flags = O_WRONLY | O_EXCL; if (query_with_output->is_outfile_append) flags |= O_APPEND; @@ -891,6 +860,74 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa } } + // Run some local checks to make sure queries into output file will work before sending to server. + try + { + if (const auto * query_with_output = dynamic_cast(parsed_query.get())) + { + String out_file; + if (query_with_output->out_file) + { + const auto & out_file_node = query_with_output->out_file->as(); + out_file = out_file_node.value.safeGet(); + + std::string compression_method_string; + + if (query_with_output->compression) + { + const auto & compression_method_node = query_with_output->compression->as(); + compression_method_string = compression_method_node.value.safeGet(); + } + + CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); + UInt64 compression_level = 3; + + if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); + } + + if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot append to compressed file. Please use uncompressed file or remove APPEND keyword."); + } + + if (query_with_output->compression_level) + { + const auto & compression_level_node = query_with_output->compression_level->as(); + bool res = compression_level_node.value.tryGet(compression_level); + auto range = getCompressionLevelRange(compression_method); + + if (!res || compression_level < range.first || compression_level > range.second) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Invalid compression level, must be positive integer in range {}-{}", + range.first, + range.second); + } + + if (fs::exists(out_file)) + { + if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::FILE_ALREADY_EXISTS, + "File {} exists, consider using APPEND or TRUNCATE.", + out_file); + } + } + } + } + } + catch (...) + { + throw LocalFormatError(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode()); + } + const auto & settings = global_context->getSettingsRef(); const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1; From 508e4906c08bf3e3c5a80335d5573cc38974b7e1 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Fri, 16 Jun 2023 01:47:28 -0700 Subject: [PATCH 054/171] Avoid double-catching the exceptions. --- src/Client/ClientBase.cpp | 93 ++++++++++++++++++--------------------- 1 file changed, 43 insertions(+), 50 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 08e14f83982..ea51c5fd0f8 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -861,72 +861,65 @@ void ClientBase::processOrdinaryQuery(const String & query_to_execute, ASTPtr pa } // Run some local checks to make sure queries into output file will work before sending to server. - try + if (const auto * query_with_output = dynamic_cast(parsed_query.get())) { - if (const auto * query_with_output = dynamic_cast(parsed_query.get())) + String out_file; + if (query_with_output->out_file) { - String out_file; - if (query_with_output->out_file) + const auto & out_file_node = query_with_output->out_file->as(); + out_file = out_file_node.value.safeGet(); + + std::string compression_method_string; + + if (query_with_output->compression) { - const auto & out_file_node = query_with_output->out_file->as(); - out_file = out_file_node.value.safeGet(); + const auto & compression_method_node = query_with_output->compression->as(); + compression_method_string = compression_method_node.value.safeGet(); + } - std::string compression_method_string; + CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); + UInt64 compression_level = 3; - if (query_with_output->compression) - { - const auto & compression_method_node = query_with_output->compression->as(); - compression_method_string = compression_method_node.value.safeGet(); - } + if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); + } - CompressionMethod compression_method = chooseCompressionMethod(out_file, compression_method_string); - UInt64 compression_level = 3; + if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Cannot append to compressed file. Please use uncompressed file or remove APPEND keyword."); + } - if (query_with_output->is_outfile_append && query_with_output->is_outfile_truncate) - { + if (query_with_output->compression_level) + { + const auto & compression_level_node = query_with_output->compression_level->as(); + bool res = compression_level_node.value.tryGet(compression_level); + auto range = getCompressionLevelRange(compression_method); + + if (!res || compression_level < range.first || compression_level > range.second) throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Cannot use INTO OUTFILE with APPEND and TRUNCATE simultaneously."); - } + "Invalid compression level, must be positive integer in range {}-{}", + range.first, + range.second); + } - if (query_with_output->is_outfile_append && compression_method != CompressionMethod::None) + if (fs::exists(out_file)) + { + if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) { throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Cannot append to compressed file. Please use uncompressed file or remove APPEND keyword."); - } - - if (query_with_output->compression_level) - { - const auto & compression_level_node = query_with_output->compression_level->as(); - bool res = compression_level_node.value.tryGet(compression_level); - auto range = getCompressionLevelRange(compression_method); - - if (!res || compression_level < range.first || compression_level > range.second) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Invalid compression level, must be positive integer in range {}-{}", - range.first, - range.second); - } - - if (fs::exists(out_file)) - { - if (!query_with_output->is_outfile_append && !query_with_output->is_outfile_truncate) - { - throw Exception( - ErrorCodes::FILE_ALREADY_EXISTS, - "File {} exists, consider using APPEND or TRUNCATE.", - out_file); - } + ErrorCodes::FILE_ALREADY_EXISTS, + "File {} exists, consider using APPEND or TRUNCATE.", + out_file); } } } } - catch (...) - { - throw LocalFormatError(getCurrentExceptionMessageAndPattern(print_stack_trace), getCurrentExceptionCode()); - } const auto & settings = global_context->getSettingsRef(); const Int32 signals_before_stop = settings.partial_result_on_first_cancel ? 2 : 1; From afa74f697cd56be60254082e2febf42943166e73 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 16 Jun 2023 19:38:50 +0000 Subject: [PATCH 055/171] Refactor a bit. --- src/Interpreters/ConcurrentHashJoin.cpp | 1 + src/Interpreters/PreparedSets.cpp | 116 ++++++-------- src/Interpreters/PreparedSets.h | 147 ++---------------- src/Interpreters/Set.cpp | 5 +- src/Interpreters/Set.h | 22 +-- src/Planner/CollectSets.cpp | 1 + src/Planner/PlannerActionsVisitor.cpp | 3 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 2 + .../MergeTree/MergeTreeSequentialSource.cpp | 1 + src/Storages/MergeTree/RPNBuilder.cpp | 14 +- src/Storages/StorageAzureBlob.cpp | 1 + src/Storages/StorageDistributed.cpp | 1 + src/Storages/StorageDummy.cpp | 1 + src/Storages/StorageFile.cpp | 1 + src/Storages/StorageMemory.cpp | 1 + src/Storages/StorageURL.cpp | 1 + .../System/StorageSystemZooKeeper.cpp | 2 + src/Storages/VirtualColumnUtils.cpp | 24 ++- 18 files changed, 111 insertions(+), 233 deletions(-) diff --git a/src/Interpreters/ConcurrentHashJoin.cpp b/src/Interpreters/ConcurrentHashJoin.cpp index 03c173a73d9..fc24f0ae029 100644 --- a/src/Interpreters/ConcurrentHashJoin.cpp +++ b/src/Interpreters/ConcurrentHashJoin.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 6df7c748e60..c79b667789b 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -1,12 +1,23 @@ #include #include #include +#include #include #include #include #include #include "Common/logger_useful.h" #include "Processors/QueryPlan/CreatingSetsStep.h" +#include "Processors/Executors/CompletedPipelineExecutor.h" +#include "Processors/QueryPlan/BuildQueryPipelineSettings.h" +#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" +#include "Processors/Sinks/EmptySink.h" +#include "Processors/Sinks/NullSink.h" +#include +#include "Core/Block.h" +#include +#include +#include namespace DB { @@ -16,6 +27,35 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +class FutureSetFromTuple final : public FutureSet +{ +public: + FutureSetFromTuple(Block block, const Settings & settings); + + bool isReady() const override { return true; } + SetPtr get() const override { return set; } + + SetPtr buildOrderedSetInplace(const ContextPtr & context) override; + + DataTypes getTypes() const override { return set->getElementsTypes(); } + +private: + SetPtr set; + SetKeyColumns set_key_columns; +}; + + +FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} +bool FutureSetFromStorage::isReady() const { return set != nullptr; } +SetPtr FutureSetFromStorage::get() const { return set; } +DataTypes FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } + +SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) +{ + return set->hasExplicitSetElements() ? set : nullptr; +} + + PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) { /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, @@ -240,8 +280,6 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) return nullptr; } - // std::cerr << "... external_table_set " << reinterpret_cast(external_table_set.get()) << std::endl; - if (external_table_set) return subquery.set = external_table_set->buildOrderedSetInplace(context); @@ -261,6 +299,13 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) return subquery.set; } +bool FutureSetFromSubquery::isReady() const { return subquery.set != nullptr && subquery.set->isCreated(); } + +std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) +{ + return buildPlan(context, false); +} + static SizeLimits getSizeLimitsForSet(const Settings & settings) { return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); @@ -271,25 +316,6 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c if (subquery.set) return nullptr; - // std::cerr << StackTrace().toString() << std::endl; - - // auto set_cache = context->getPreparedSetsCache(); - // if (set_cache) - // { - // auto from_cache = set_cache->findOrPromiseToBuild(subquery.key); - // if (from_cache.index() == 0) - // { - // subquery.promise_to_fill_set = std::move(std::get<0>(from_cache)); - // } - // else - // { - // LOG_TRACE(&Poco::Logger::get("FutureSetFromSubquery"), "Waiting for set, key: {}", subquery.key); - // set = std::get<1>(from_cache).get(); - // return nullptr; - // } - // } - - const auto & settings = context->getSettingsRef(); auto size_limits = getSizeLimitsForSet(settings); @@ -298,10 +324,6 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c auto plan = subquery.detachSource(); auto description = subquery.key; - // WriteBufferFromOwnString buf; - // plan->explainPlan(buf, {.header=true}); - // std::cerr << buf.str() << std::endl; - auto creating_set = std::make_unique( plan->getCurrentDataStream(), description, @@ -314,20 +336,6 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c return plan; } -// static SizeLimits getSizeLimitsForOrderedSet(const Settings & settings) -// { -// if (settings.use_index_for_in_with_subqueries_max_values && -// settings.use_index_for_in_with_subqueries_max_values < settings.max_rows_in_set) -// return getSizeLimitsForUnorderedSet(settings); - -// return SizeLimits(settings.use_index_for_in_with_subqueries_max_values, settings.max_bytes_in_set, OverflowMode::BREAK); -// } - -// SizeLimits FutureSet::getSizeLimitsForSet(const Settings & settings, bool ordered_set) -// { -// return ordered_set ? getSizeLimitsForOrderedSet(settings) : getSizeLimitsForUnorderedSet(settings); -// } - FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) { bool create_ordered_set = false; @@ -342,10 +350,8 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set_key_columns.filter = ColumnUInt8::create(block.rows()); - //set->initSetElements(); set->insertFromColumns(columns, set_key_columns); set->finishInsert(); - //block(std::move(block_)) } FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_) @@ -359,8 +365,6 @@ DataTypes FutureSetFromSubquery::getTypes() const return Set::getElementTypes(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName(), transform_null_in); } -FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} - SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) { if (set->hasExplicitSetElements()) @@ -378,30 +382,4 @@ SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) return set; } -std::unique_ptr FutureSetFromTuple::build(const ContextPtr &) -{ - // const auto & settings = context->getSettingsRef(); - // auto size_limits = getSizeLimitsForSet(settings, false); - // fill(size_limits, settings.transform_null_in, false); - return nullptr; -} - -// void FutureSetFromTuple::buildForTuple(SizeLimits size_limits, bool transform_null_in) -// { -// fill(size_limits, transform_null_in, false); -// } - -// void FutureSetFromTuple::fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set) -// { -// //std::cerr << StackTrace().toString() << std::endl; - -// if (set) -// return; - -// set = std::make_shared(size_limits, create_ordered_set, transform_null_in); -// set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); -// set->insertFromBlock(block.getColumnsWithTypeAndName()); -// set->finishInsert(); -// } - }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index a119c24ad10..f034aca747f 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -2,23 +2,13 @@ #include #include -#include #include #include #include -#include +#include #include -#include -#include -#include "Core/Block.h" -#include "Interpreters/Context.h" -#include "Interpreters/Set.h" -#include "Processors/Executors/CompletedPipelineExecutor.h" -#include "Processors/QueryPlan/BuildQueryPipelineSettings.h" -#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" -#include "Processors/Sinks/EmptySink.h" -#include "Processors/Sinks/NullSink.h" -#include +#include +#include namespace DB { @@ -32,80 +22,44 @@ class InterpreterSelectWithUnionQuery; class IQueryTreeNode; using QueryTreeNodePtr = std::shared_ptr; +struct Settings; + /// Represents a set in a query that might be referenced at analysis time and built later during execution. /// Also it can represent a constant set that is ready to use. /// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there /// must be an explicit step to build them before they can be used. -/// FutureSet objects can be stored in PreparedSets and are not intended to be used from multiple threads. -// class FutureSet final -// { -// public: -// FutureSet() = default; - -// /// Create FutureSet from an object that will be created in the future. -// explicit FutureSet(const std::shared_future & future_set_) : future_set(future_set_) {} - -// /// Create FutureSet from a ready set. -// explicit FutureSet(SetPtr readySet); - -// /// The set object will be ready in the future, as opposed to 'null' object when FutureSet is default constructed. -// bool isValid() const { return future_set.valid(); } - -// /// The the value of SetPtr is ready, but the set object might not have been filled yet. -// bool isReady() const; - -// /// The set object is ready and filled. -// bool isCreated() const; - -// SetPtr get() const { chassert(isReady()); return future_set.get(); } - -// private: -// std::shared_future future_set; -// }; - +/// Set may be useful for indexes, in this case special ordered set with stored elements is build inplace. class FutureSet { public: virtual ~FutureSet() = default; virtual bool isReady() const = 0; - virtual bool isFilled() const = 0; virtual SetPtr get() const = 0; virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; - virtual std::unique_ptr build(const ContextPtr & context) = 0; virtual DataTypes getTypes() const = 0; - - // static SizeLimits getSizeLimitsForSet(const Settings & settings, bool ordered_set); }; using FutureSetPtr = std::shared_ptr; -class FutureSetFromTuple final : public FutureSet + +class FutureSetFromStorage : public FutureSet { public: - FutureSetFromTuple(Block block, const Settings & settings); + FutureSetFromStorage(SetPtr set_); - bool isReady() const override { return true; } - bool isFilled() const override { return true; } - SetPtr get() const override { return set; } - - SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - - std::unique_ptr build(const ContextPtr &) override; - - DataTypes getTypes() const override { return set->getElementsTypes(); } - -/// void buildForTuple(SizeLimits size_limits, bool transform_null_in); + bool isReady() const override; + SetPtr get() const override; + SetPtr buildOrderedSetInplace(const ContextPtr &) override; + DataTypes getTypes() const override; private: SetPtr set; - Set::SetKeyColumns set_key_columns; - - //void fill(SizeLimits size_limits, bool transform_null_in, bool create_ordered_set); }; + /// Information on how to build set for the [GLOBAL] IN section. class SubqueryForSet { @@ -122,11 +76,6 @@ public: /// Build this set from the result of the subquery. String key; SetPtr set; - /// After set_in_progress is finished it will be put into promise_to_fill_set and thus all FutureSet's - /// that are referencing this set will be filled. - - std::promise promise_to_fill_set; - // FutureSet set = FutureSet{promise_to_fill_set.get_future()}; /// If set, put the result into the table. /// This is a temporary table for transferring to remote servers for distributed query processing. @@ -142,25 +91,18 @@ class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_t public: FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_); - bool isReady() const override { return subquery.set != nullptr && subquery.set->isCreated(); } - bool isFilled() const override { return isReady(); } + bool isReady() const override; SetPtr get() const override { return subquery.set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - std::unique_ptr build(const ContextPtr & context) override - { - return buildPlan(context, false); - } + std::unique_ptr build(const ContextPtr & context); DataTypes getTypes() const override; - // void addStorage(StoragePtr storage) { subquery.table = std::move(storage); } - SubqueryForSet & getSubquery() { return subquery; } private: - //SetPtr set; SubqueryForSet subquery; FutureSetPtr external_table_set; bool transform_null_in; @@ -168,52 +110,6 @@ private: std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); }; -class FutureSetFromStorage : public FutureSet -{ -public: - FutureSetFromStorage(SetPtr set_); // : set(std::move(set_) {} - - bool isReady() const override { return set != nullptr; } - bool isFilled() const override { return isReady(); } - SetPtr get() const override { return set; } - - SetPtr buildOrderedSetInplace(const ContextPtr &) override - { - return set->hasExplicitSetElements() ? set : nullptr; - } - - DataTypes getTypes() const override { return set->getElementsTypes(); } - - std::unique_ptr build(const ContextPtr &) override { return nullptr; } - -private: - SetPtr set; -}; - -// class FutureSetFromFuture : public FutureSet -// { -// public: -// FutureSetFromFuture(std::shared_future future_set_); - -// bool isReady() const override { return future_set.wait_for(std::chrono::seconds(0)) == std::future_status::ready; } -// SetPtr get() const override { return future_set.get(); } - -// SetPtr buildOrderedSetInplace(const ContextPtr &) override -// { -// fill(true); -// return set; -// } - -// std::unique_ptr build(const ContextPtr &) override -// { -// fill(false); -// return nullptr; -// } - -// private: -// std::shared_future future_set; -// } - struct PreparedSetKey { using Hash = std::pair; @@ -250,28 +146,17 @@ public: }; using SubqueriesForSets = std::vector; - // SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, - // SizeLimits set_size_limit, bool transform_null_in); - FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); - //void addStorageToSubquery(const String & subquery_id, StoragePtr external_storage); - FutureSetPtr getFuture(const PreparedSetKey & key) const; - //SubqueryForSet & getSubquery(const String & subquery_id); - // SetPtr get(const PreparedSetKey & key) const; /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. /// SetPtr would still be available for consumers of PreparedSets. SubqueriesForSets detachSubqueries(); - /// Returns all sets that match the given ast hash not checking types - /// Used in KeyCondition and MergeTreeIndexConditionBloomFilter to make non exact match for types in PreparedSetKey - //std::vector getByTreeHash(IAST::Hash ast_hash) const; - const std::unordered_map & getSets() const { return sets; } bool empty() const; diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 52e9a3a9bcb..83005f3d3a7 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -265,7 +265,10 @@ bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder) void Set::appendSetElements(SetKeyColumns & holder) { - //std::cerr << "========= " << keys_size << ' ' << holder.key_columns.size() << std::endl; + if (holder.key_columns.size() != keys_size || set_elements.size() != keys_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of key columns for set. Expected {} got {} and {}", + keys_size, holder.key_columns.size(), set_elements.size()); + size_t rows = holder.key_columns.at(0)->size(); for (size_t i = 0; i < keys_size; ++i) { diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 9b76ab30a1b..a6908d2bdd3 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -21,6 +21,17 @@ using FunctionBasePtr = std::shared_ptr; class Chunk; +/// Prepared key columns for set which can be added to fill set elements. +/// Used only to upgrade set from tuple. +struct SetKeyColumns +{ + /// The constant columns to the right of IN are not supported directly. For this, they first materialize. + ColumnRawPtrs key_columns; + Columns materialized_columns; + ColumnPtr null_map_holder; + ColumnUInt8::MutablePtr filter; +}; + /** Data structure for implementation of IN expression. */ class Set @@ -48,17 +59,6 @@ public: bool insertFromColumns(const Columns & columns); bool insertFromBlock(const ColumnsWithTypeAndName & columns); - - struct SetKeyColumns - { - //ColumnRawPtrs key_columns; - /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - ColumnRawPtrs key_columns; - Columns materialized_columns; - ColumnPtr null_map_holder; - ColumnUInt8::MutablePtr filter; - }; - void initSetElements(); bool insertFromColumns(const Columns & columns, SetKeyColumns & holder); void appendSetElements(SetKeyColumns & holder); diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 68ad1ab78d3..84ab453c065 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include namespace DB diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index bbac9bf5c4b..087cd3000b9 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -17,6 +17,7 @@ #include #include +#include #include #include @@ -663,7 +664,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma column.name = planner_context->createSetKey(in_second_argument); column.type = std::make_shared(); - bool set_is_created = set->isFilled(); + bool set_is_created = set->isReady(); auto column_set = ColumnSet::create(1, std::move(set)); if (set_is_created) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 0909ee9f1eb..da27e7b1293 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -1,6 +1,8 @@ #include #include #include +//#include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index e27354f9d16..5a6d59bf0be 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index b82c350fa1a..1acc3ec6ce9 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -9,6 +9,7 @@ #include #include +#include #include #include @@ -301,26 +302,15 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const auto hash = ast_node->getTreeHash(); auto key = PreparedSetKey::forSubquery(hash); - // std::cerr << ".........Getting from AST \n" << ast_node->dumpTree() << std::endl - // << key.toString() << std::endl; - for (const auto & [k, v] : prepared_sets->getSets()) { - // std::cerr << "........... " << k.toString() << std::endl; if (k.ast_hash == hash) return v; } - - //return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); } else if (dag_node) { - - // std::cerr << "...........Getting from DAG\n"; const auto * node_without_alias = getNodeWithoutAlias(dag_node); - // std::cerr << ".......... node_without_alias : " << node_without_alias->result_name - // << ' ' << node_without_alias->result_type->getName() - // << ' ' << (node_without_alias->column ? node_without_alias->column->getName() : "") << std::endl; return tryGetSetFromDAGNode(node_without_alias); } @@ -369,8 +359,6 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( auto lhs = removeNullable(recursiveRemoveLowCardinality(data_types[i])); auto rhs = removeNullable(recursiveRemoveLowCardinality(set_types[indexes_mapping[i].tuple_index])); - // std::cerr << "============ " << lhs->getName() << ' ' << rhs->getName() << std::endl; - if (!lhs->equals(*rhs)) return false; } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 3ee176a68b7..b80f41543b5 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -13,6 +13,7 @@ #include #include +#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 042492aba32..763dbac04cf 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -91,6 +91,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/StorageDummy.cpp b/src/Storages/StorageDummy.cpp index 4f2fb3883bf..e2396a54acb 100644 --- a/src/Storages/StorageDummy.cpp +++ b/src/Storages/StorageDummy.cpp @@ -5,6 +5,7 @@ #include #include +#include namespace DB { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 72347789790..b3359dd3427 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -23,6 +23,7 @@ #include #include +#include #include #include #include diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index b7772d00776..6ed0583bd44 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 520576d3961..2e72c957a24 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -39,6 +39,7 @@ #include #include #include +#include namespace DB diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 3f6575cb78d..d1c978d3469 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -15,12 +15,14 @@ #include #include #include +#include #include #include #include #include #include #include +#include #include #include #include diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index be2206a78e9..58d00216dc7 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -20,6 +20,13 @@ #include #include +#include +#include +#include +#include +#include +#include + #include #include #include @@ -215,15 +222,18 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex if (column_set) { auto future_set = column_set->getData(); - if (!future_set->isFilled()) + if (!future_set->isReady()) { - auto plan = future_set->build(context); - auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); + if (auto * set_from_subquery = typeid_cast(future_set.get())) + { + auto plan = set_from_subquery->build(context); + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); - CompletedPipelineExecutor executor(pipeline); - executor.execute(); + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + } } } } From 83e2196a171b21840a03dad6154d9d350d7dd00a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 12:26:53 +0000 Subject: [PATCH 056/171] Make is_shared const --- src/Interpreters/Cache/QueryCache.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index bafa78f13d5..4bc33e36441 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -49,7 +49,7 @@ public: /// If the associated entry can be read by other users. In general, sharing is a bad idea: First, it is unlikely that different /// users pose the same queries. Second, sharing potentially breaches security. E.g. User A should not be able to bypass row /// policies on some table by running the same queries as user B for whom no row policies exist. - bool is_shared; + const bool is_shared; /// When does the entry expire? const std::chrono::time_point expires_at; From 840e73eb7444400514463885de75b6d551935481 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 12:27:44 +0000 Subject: [PATCH 057/171] Update #includes --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index de2e2b9ad92..fa01b1da67c 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include #include @@ -14,7 +15,6 @@ #include #include #include -#include #include #include From 0adc98865d28fbffc5d6aff7def6d8cfb16fb91f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 12:43:14 +0000 Subject: [PATCH 058/171] Add variable for access to found key components --- src/Interpreters/Cache/QueryCache.cpp | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 3118f386111..1afb538c01d 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -385,19 +385,22 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar return; } - if (!entry->key.is_shared && entry->key.user_name != key.user_name) + const auto & entry_key = entry->key; + const auto & entry_mapped = entry->mapped; + + if (!entry_key.is_shared && entry_key.user_name != key.user_name) { LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.queryStringFromAst()); return; } - if (IsStale()(entry->key)) + if (IsStale()(entry_key)) { LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.queryStringFromAst()); return; } - if (!entry->key.is_compressed) + if (!entry_key.is_compressed) { // Cloning chunks isn't exactly great. It could be avoided by another indirection, i.e. wrapping Entry's members chunks, totals and // extremes into shared_ptrs and assuming that the lifecycle of these shared_ptrs coincides with the lifecycle of the Entry @@ -406,15 +409,15 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar // optimization. Chunks cloned_chunks; - for (const auto & chunk : entry->mapped->chunks) + for (const auto & chunk : entry_mapped->chunks) cloned_chunks.push_back(chunk.clone()); - buildSourceFromChunks(entry->key.header, std::move(cloned_chunks), entry->mapped->totals, entry->mapped->extremes); + buildSourceFromChunks(entry_key.header, std::move(cloned_chunks), entry_mapped->totals, entry_mapped->extremes); } else { Chunks decompressed_chunks; - const Chunks & chunks = entry->mapped->chunks; + const Chunks & chunks = entry_mapped->chunks; for (const auto & chunk : chunks) { const Columns & columns = chunk.getColumns(); @@ -428,7 +431,7 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar decompressed_chunks.push_back(std::move(decompressed_chunk)); } - buildSourceFromChunks(entry->key.header, std::move(decompressed_chunks), entry->mapped->totals, entry->mapped->extremes); + buildSourceFromChunks(entry_key.header, std::move(decompressed_chunks), entry_mapped->totals, entry_mapped->extremes); } LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.queryStringFromAst()); From 6e8af092897162035c0add81ef2d8d0028cff810 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 12:48:29 +0000 Subject: [PATCH 059/171] Provide query cache reader with dummy block --- src/Interpreters/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index fa01b1da67c..5cf86412c3e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -734,7 +734,7 @@ static std::tuple executeQueryImpl( && res.pipeline.pulling()) { QueryCache::Key key( - ast, res.pipeline.getHeader(), + ast, /*dummy for header*/ {}, context->getUserName(), /*dummy for is_shared*/ false, /*dummy value for expires_at*/ std::chrono::system_clock::from_time_t(1), /*dummy value for is_compressed*/ false); From 69e7c1cc82944305e5019630674bacab5170b319 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 18:22:15 +0000 Subject: [PATCH 060/171] Introduce convenience ctor for Key construction during read --- src/Interpreters/Cache/QueryCache.cpp | 5 +++++ src/Interpreters/Cache/QueryCache.h | 4 ++++ src/Interpreters/executeQuery.cpp | 6 +----- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 1afb538c01d..e2cf5a7d6dd 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -132,6 +132,11 @@ QueryCache::Key::Key( { } +QueryCache::Key::Key(ASTPtr ast_, const String & user_name_) + : QueryCache::Key(ast_, {}, user_name_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST or user name +{ +} + bool QueryCache::Key::operator==(const Key & other) const { return ast->getTreeHash() == other.ast->getTreeHash(); diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 4bc33e36441..6ef7cc60918 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -58,12 +58,16 @@ public: /// (we could theoretically apply compression also to the totals and extremes but it's an obscure use case) const bool is_compressed; + /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, Block header_, const String & user_name_, bool is_shared_, std::chrono::time_point expires_at_, bool is_compressed); + /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). + Key(ASTPtr ast_, const String & user_name_); + bool operator==(const Key & other) const; String queryStringFromAst() const; }; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 5cf86412c3e..b45ccf63be6 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -733,11 +733,7 @@ static std::tuple executeQueryImpl( && (can_use_query_cache && settings.enable_reads_from_query_cache) && res.pipeline.pulling()) { - QueryCache::Key key( - ast, /*dummy for header*/ {}, - context->getUserName(), /*dummy for is_shared*/ false, - /*dummy value for expires_at*/ std::chrono::system_clock::from_time_t(1), - /*dummy value for is_compressed*/ false); + QueryCache::Key key(ast, context->getUserName()); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { From 24717ed908688402d023b933f8b635f3439870fb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 18:24:20 +0000 Subject: [PATCH 061/171] Factorize check if query cache exists --- src/Interpreters/executeQuery.cpp | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index b45ccf63be6..cb68cacebda 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -645,7 +645,8 @@ static std::tuple executeQueryImpl( } } - bool can_use_query_cache = settings.use_query_cache && !internal && !ast->as(); + QueryCachePtr query_cache = context->getQueryCache(); + const bool can_use_query_cache = query_cache != nullptr && settings.use_query_cache && !internal && !ast->as(); if (!async_insert) { @@ -727,10 +728,8 @@ static std::tuple executeQueryImpl( /// - passive (read) use of the query cache is enabled, and /// - the query cache knows the query result /// then replace the pipeline by a new pipeline with a single source that is populated from the query cache - auto query_cache = context->getQueryCache(); bool read_result_from_query_cache = false; /// a query must not read from *and* write to the query cache at the same time - if (query_cache != nullptr - && (can_use_query_cache && settings.enable_reads_from_query_cache) + if (can_use_query_cache && settings.enable_reads_from_query_cache && res.pipeline.pulling()) { QueryCache::Key key(ast, context->getUserName()); @@ -749,7 +748,6 @@ static std::tuple executeQueryImpl( /// - active (write) use of the query cache is enabled /// then add a processor on top of the pipeline which stores the result in the query cache. if (!read_result_from_query_cache - && query_cache != nullptr && can_use_query_cache && settings.enable_writes_to_query_cache && res.pipeline.pulling() && (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions)) @@ -927,6 +925,7 @@ static std::tuple executeQueryImpl( context, ast, my_can_use_query_cache = can_use_query_cache, + query_cache = query_cache, enable_writes_to_query_cache = settings.enable_writes_to_query_cache, query_cache_store_results_of_queries_with_nondeterministic_functions = settings.query_cache_store_results_of_queries_with_nondeterministic_functions, log_queries, @@ -941,9 +940,7 @@ static std::tuple executeQueryImpl( { /// If active (write) use of the query cache is enabled and the query is eligible for result caching, then store the query /// result buffered in the special-purpose cache processor (added on top of the pipeline) into the cache. - auto query_cache = context->getQueryCache(); - if (query_cache != nullptr - && pulling_pipeline + if (pulling_pipeline && my_can_use_query_cache && enable_writes_to_query_cache && (!astContainsNonDeterministicFunctions(ast, context) || query_cache_store_results_of_queries_with_nondeterministic_functions)) { From 5311ebe1d295939bfd576e39cb31734e0f4f42d1 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 20:12:51 +0000 Subject: [PATCH 062/171] Make query cache work effective with nested queries. --- src/Interpreters/executeQuery.cpp | 227 +++++++++--------- .../System/StorageSystemQueryCache.cpp | 2 +- ...494_query_cache_nested_query_bug.reference | 2 + .../02494_query_cache_nested_query_bug.sh | 22 ++ 4 files changed, 135 insertions(+), 118 deletions(-) create mode 100644 tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference create mode 100755 tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index cb68cacebda..c52dab722c9 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -646,91 +646,16 @@ static std::tuple executeQueryImpl( } QueryCachePtr query_cache = context->getQueryCache(); - const bool can_use_query_cache = query_cache != nullptr && settings.use_query_cache && !internal && !ast->as(); + const bool can_use_query_cache = query_cache != nullptr && settings.use_query_cache && !internal && (ast->as() || ast->as()); + bool write_into_query_cache = false; if (!async_insert) { - /// 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 && !ast->as()) + /// If it is a non-internal SELECT, and passive/read use of the query cache is enabled, and the cache knows the query, then set + /// a pipeline with a source populated by the query cache. + auto get_result_from_query_cache = [&]() { - 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; - } - } - - interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); - - 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 ({})", ast->getID()); - - } - - if (!interpreter->ignoreQuota() && !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 (!interpreter->ignoreLimits()) - { - limits.mode = LimitsMode::LIMITS_CURRENT; - limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); - } - - if (auto * insert_interpreter = typeid_cast(&*interpreter)) - { - /// Save insertion table (not table function). TODO: support remote() table function. - auto table_id = insert_interpreter->getDatabaseTable(); - if (!table_id.empty()) - context->setInsertionTable(std::move(table_id)); - - if (insert_data_buffer_holder) - insert_interpreter->addBuffer(std::move(insert_data_buffer_holder)); - } - - { - 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(class_name + "::execute()"); - } - - res = interpreter->execute(); - - /// If - /// - it is a SELECT query, - /// - passive (read) use of the query cache is enabled, and - /// - the query cache knows the query result - /// then replace the pipeline by a new pipeline with a single source that is populated from the query cache - bool read_result_from_query_cache = false; /// a query must not read from *and* write to the query cache at the same time - if (can_use_query_cache && settings.enable_reads_from_query_cache - && res.pipeline.pulling()) + if (can_use_query_cache && settings.enable_reads_from_query_cache) { QueryCache::Key key(ast, context->getUserName()); QueryCache::Reader reader = query_cache->createReader(key); @@ -739,39 +664,114 @@ static std::tuple executeQueryImpl( QueryPipeline pipeline; pipeline.readFromQueryCache(reader.getSource(), reader.getSourceTotals(), reader.getSourceExtremes()); res.pipeline = std::move(pipeline); - read_result_from_query_cache = true; + return true; } } + return false; + }; - /// If - /// - it is a SELECT query, and - /// - active (write) use of the query cache is enabled - /// then add a processor on top of the pipeline which stores the result in the query cache. - if (!read_result_from_query_cache - && can_use_query_cache && settings.enable_writes_to_query_cache - && res.pipeline.pulling() - && (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions)) + if (!get_result_from_query_cache()) + { + /// 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 && !ast->as()) { - QueryCache::Key key( - ast, res.pipeline.getHeader(), - context->getUserName(), settings.query_cache_share_between_users, - std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), - settings.query_cache_compress_entries); - - const size_t num_query_runs = query_cache->recordQueryRun(key); - if (num_query_runs > settings.query_cache_min_query_runs) + try { - auto query_cache_writer = std::make_shared(query_cache->createWriter( - key, - std::chrono::milliseconds(settings.query_cache_min_query_duration.totalMilliseconds()), - settings.query_cache_squash_partial_results, - settings.max_block_size, - settings.query_cache_max_size_in_bytes, - settings.query_cache_max_entries)); - res.pipeline.writeResultIntoQueryCache(query_cache_writer); + 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; } } + interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal)); + + 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 ({})", ast->getID()); + + } + + if (!interpreter->ignoreQuota() && !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 (!interpreter->ignoreLimits()) + { + limits.mode = LimitsMode::LIMITS_CURRENT; + limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); + } + + if (auto * insert_interpreter = typeid_cast(&*interpreter)) + { + /// Save insertion table (not table function). TODO: support remote() table function. + auto table_id = insert_interpreter->getDatabaseTable(); + if (!table_id.empty()) + context->setInsertionTable(std::move(table_id)); + + if (insert_data_buffer_holder) + insert_interpreter->addBuffer(std::move(insert_data_buffer_holder)); + } + + { + 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(class_name + "::execute()"); + } + + res = interpreter->execute(); + + /// If it is a non-internal SELECT query, and active/write use of the query cache is enabled, then add a processor on + /// top of the pipeline which stores the result in the query cache. + if (can_use_query_cache && settings.enable_writes_to_query_cache + && (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions)) + { + QueryCache::Key key( + ast, res.pipeline.getHeader(), + context->getUserName(), settings.query_cache_share_between_users, + std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), + settings.query_cache_compress_entries); + + const size_t num_query_runs = query_cache->recordQueryRun(key); + if (num_query_runs > settings.query_cache_min_query_runs) + { + auto query_cache_writer = std::make_shared(query_cache->createWriter( + key, + std::chrono::milliseconds(settings.query_cache_min_query_duration.totalMilliseconds()), + settings.query_cache_squash_partial_results, + settings.max_block_size, + settings.query_cache_max_size_in_bytes, + settings.query_cache_max_entries)); + res.pipeline.writeResultIntoQueryCache(query_cache_writer); + write_into_query_cache = true; + } + } + + } } } @@ -924,10 +924,7 @@ static std::tuple executeQueryImpl( auto finish_callback = [elem, context, ast, - my_can_use_query_cache = can_use_query_cache, - query_cache = query_cache, - enable_writes_to_query_cache = settings.enable_writes_to_query_cache, - query_cache_store_results_of_queries_with_nondeterministic_functions = settings.query_cache_store_results_of_queries_with_nondeterministic_functions, + write_into_query_cache, log_queries, log_queries_min_type = settings.log_queries_min_type, log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(), @@ -938,14 +935,10 @@ static std::tuple executeQueryImpl( pulling_pipeline = pipeline.pulling(), query_span](QueryPipeline & query_pipeline) mutable { - /// If active (write) use of the query cache is enabled and the query is eligible for result caching, then store the query - /// result buffered in the special-purpose cache processor (added on top of the pipeline) into the cache. - if (pulling_pipeline - && my_can_use_query_cache && enable_writes_to_query_cache - && (!astContainsNonDeterministicFunctions(ast, context) || query_cache_store_results_of_queries_with_nondeterministic_functions)) - { + if (write_into_query_cache) + /// 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(); - } QueryStatusPtr process_list_elem = context->getProcessListElement(); diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 3dfc5cf298a..117fb4e8a5c 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -29,7 +29,7 @@ StorageSystemQueryCache::StorageSystemQueryCache(const StorageID & table_id_) void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { - auto query_cache = context->getQueryCache(); + QueryCachePtr query_cache = context->getQueryCache(); if (!query_cache) return; diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference new file mode 100644 index 00000000000..389e2621455 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference @@ -0,0 +1,2 @@ +2 +0 diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh new file mode 100755 index 00000000000..f6fb15f76eb --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +# Tags: no-parallel +# Tag no-parallel: Messes with internal cache + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# Start with empty query cache (QC). +${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" + +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS tab" +${CLICKHOUSE_CLIENT} --query "CREATE TABLE tab (a UInt64) ENGINE=MergeTree() ORDER BY a" +${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (1) (2) (3)" +${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (3) (4) (5)" + +# Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated +# with EXPLAIN PLAN, we need need to check the logs. +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1" 2>&1 | grep "Aggregated. " | wc -l + +${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" From b5b67f809495d4dc71f7bfbffccc41dd828c2f7a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sat, 17 Jun 2023 21:11:09 +0000 Subject: [PATCH 063/171] Exclude fasttest --- tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index f6fb15f76eb..f451e9b0709 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest # Tag no-parallel: Messes with internal cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 880c9aea09d737489e274c6ff05f12c5ab072c63 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Jun 2023 10:47:43 +0000 Subject: [PATCH 064/171] Fixing style. --- src/Interpreters/PreparedSets.cpp | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index c79b667789b..3c5e930a6cf 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -2,19 +2,17 @@ #include #include #include -#include #include -#include #include -#include "Common/logger_useful.h" -#include "Processors/QueryPlan/CreatingSetsStep.h" -#include "Processors/Executors/CompletedPipelineExecutor.h" -#include "Processors/QueryPlan/BuildQueryPipelineSettings.h" -#include "Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h" -#include "Processors/Sinks/EmptySink.h" -#include "Processors/Sinks/NullSink.h" +#include +#include +#include +#include +#include +#include +#include #include -#include "Core/Block.h" +#include #include #include #include From f3398f9908214f8d5ab06c5c625f2cf9f006364b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Jun 2023 12:17:52 +0000 Subject: [PATCH 065/171] Try to make more consisten type check for sets. --- src/Storages/MergeTree/RPNBuilder.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 1acc3ec6ce9..d7dd7ff3654 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -384,8 +384,11 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (node_without_alias->column) { auto future_set = tryGetSetFromDAGNode(node_without_alias); + if (const auto * from_subquery = typeid_cast(future_set.get())) + return future_set; + auto set_types = future_set->getTypes(); - if (types_match(future_set->getTypes())) + if (types_match(set_types)) return future_set; } } From 8e7d06e0a4211de261a3aa9ef48561df30802403 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Jun 2023 12:56:24 +0000 Subject: [PATCH 066/171] Remove isReady from FutureSet iface. --- src/Functions/in.cpp | 8 ++------ src/Interpreters/ExpressionActions.cpp | 11 ++++------- src/Interpreters/PreparedSets.cpp | 11 +++++++---- src/Interpreters/PreparedSets.h | 6 +----- src/Planner/PlannerActionsVisitor.cpp | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 4 ++-- src/Storages/MergeTree/RPNBuilder.cpp | 6 +----- src/Storages/VirtualColumnUtils.cpp | 2 +- 8 files changed, 19 insertions(+), 31 deletions(-) diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index 0e576b92aad..c19d1e72003 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -123,14 +123,10 @@ public: } auto future_set = column_set->getData(); - if (!future_set || !future_set->isReady()) + auto set = future_set ? future_set->get() : nullptr; + if (!future_set || !set) throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName()); - // if (auto * for_tuple = typeid_cast(future_set.get())) - // if (!for_tuple->isReady()) - // for_tuple->buildForTuple(size_limits, transform_null_in); - - auto set = future_set->get(); auto set_types = set->getDataTypes(); if (tuple && set_types.size() != 1 && set_types.size() == tuple->tupleSize()) diff --git a/src/Interpreters/ExpressionActions.cpp b/src/Interpreters/ExpressionActions.cpp index 989fa12cba0..f1c577948eb 100644 --- a/src/Interpreters/ExpressionActions.cpp +++ b/src/Interpreters/ExpressionActions.cpp @@ -936,15 +936,12 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con for (const auto & action : actions) { if (action.node->type == ActionsDAG::ActionType::COLUMN && action.node->result_name == set_to_check) - { // Constant ColumnSet cannot be empty, so we only need to check non-constant ones. if (const auto * column_set = checkAndGetColumn(action.node->column.get())) - { - auto set = column_set->getData(); - if (set && set->isReady() && set->get()->getTotalRowCount() == 0) - return true; - } - } + if (auto future_set = column_set->getData()) + if (auto set = future_set->get()) + if (set->getTotalRowCount() == 0) + return true; } } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 3c5e930a6cf..95cfa57857b 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -30,9 +30,7 @@ class FutureSetFromTuple final : public FutureSet public: FutureSetFromTuple(Block block, const Settings & settings); - bool isReady() const override { return true; } SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr & context) override; DataTypes getTypes() const override { return set->getElementsTypes(); } @@ -44,7 +42,6 @@ private: FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} -bool FutureSetFromStorage::isReady() const { return set != nullptr; } SetPtr FutureSetFromStorage::get() const { return set; } DataTypes FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } @@ -297,7 +294,13 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) return subquery.set; } -bool FutureSetFromSubquery::isReady() const { return subquery.set != nullptr && subquery.set->isCreated(); } +SetPtr FutureSetFromSubquery::get() const +{ + if (subquery.set != nullptr && subquery.set->isCreated()) + return subquery.set; + + return nullptr; +} std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) { diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index f034aca747f..1e04bbbe3e8 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -34,9 +34,7 @@ class FutureSet public: virtual ~FutureSet() = default; - virtual bool isReady() const = 0; virtual SetPtr get() const = 0; - virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; virtual DataTypes getTypes() const = 0; @@ -50,7 +48,6 @@ class FutureSetFromStorage : public FutureSet public: FutureSetFromStorage(SetPtr set_); - bool isReady() const override; SetPtr get() const override; SetPtr buildOrderedSetInplace(const ContextPtr &) override; DataTypes getTypes() const override; @@ -91,8 +88,7 @@ class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_t public: FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_); - bool isReady() const override; - SetPtr get() const override { return subquery.set; } + SetPtr get() const override; SetPtr buildOrderedSetInplace(const ContextPtr & context) override; diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 087cd3000b9..49312e5c4ad 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -664,7 +664,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma column.name = planner_context->createSetKey(in_second_argument); column.type = std::make_shared(); - bool set_is_created = set->isReady(); + bool set_is_created = set->get() != nullptr; auto column_set = ColumnSet::create(1, std::move(set)); if (set_is_created) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index da27e7b1293..751a1f95b7b 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -137,7 +137,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets for (auto & [description, future_set] : subqueries_for_sets) { - if (future_set->isReady()) + if (future_set->get()) continue; auto plan = future_set->build(context); @@ -172,7 +172,7 @@ std::vector> DelayedCreatingSetsStep::makePlansForSet for (auto & [description, future_set] : step.subqueries_for_sets) { - if (future_set->isReady()) + if (future_set->get()) continue; auto plan = future_set->build(step.context); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index d7dd7ff3654..ad3fb984ce7 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -384,11 +384,7 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (node_without_alias->column) { auto future_set = tryGetSetFromDAGNode(node_without_alias); - if (const auto * from_subquery = typeid_cast(future_set.get())) - return future_set; - - auto set_types = future_set->getTypes(); - if (types_match(set_types)) + if (types_match(future_set->getTypes())) return future_set; } } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 58d00216dc7..b83afe766aa 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -222,7 +222,7 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex if (column_set) { auto future_set = column_set->getData(); - if (!future_set->isReady()) + if (!future_set->get()) { if (auto * set_from_subquery = typeid_cast(future_set.get())) { From 7280078bc6198647e8e695b8658beadc0e864ce2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 19 Jun 2023 19:01:56 +0000 Subject: [PATCH 067/171] Refactoring a bit more. --- src/Interpreters/ActionsVisitor.cpp | 15 +- src/Interpreters/GlobalSubqueriesVisitor.h | 4 +- src/Interpreters/PreparedSets.cpp | 208 ++++++++++++------ src/Interpreters/PreparedSets.h | 91 ++++---- src/Interpreters/Set.h | 2 + src/Planner/CollectSets.cpp | 11 +- src/Planner/Planner.cpp | 8 +- src/Planner/PlannerActionsVisitor.cpp | 7 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 10 +- src/Processors/QueryPlan/CreatingSetsStep.h | 8 +- .../Optimizations/filterPushDown.cpp | 2 +- .../Transforms/CreatingSetsTransform.cpp | 10 +- src/Storages/KVStorageUtils.cpp | 13 +- src/Storages/MergeTree/RPNBuilder.cpp | 30 +-- 14 files changed, 248 insertions(+), 171 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 4b38f8c9434..91c490f8a4c 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -449,8 +449,8 @@ FutureSetPtr makeExplicitSet( if (const auto * low_cardinality_type = typeid_cast(element_type.get())) element_type = low_cardinality_type->getDictionaryType(); - auto set_key = PreparedSetKey::forLiteral(right_arg->getTreeHash(), set_element_types); - if (auto set = prepared_sets.getFuture(set_key)) + auto set_key = right_arg->getTreeHash(); + if (auto set = prepared_sets.find(set_key, set_element_types)) return set; /// Already prepared. Block block; @@ -1397,18 +1397,17 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool return {}; //std::cerr << queryToString(right_in_operand) << std::endl; - PreparedSetKey set_key; + PreparedSets::Hash set_key; if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery()); auto query_tree = interpreter.getQueryTree(); if (auto * query_node = query_tree->as()) query_node->setIsSubquery(true); - // std::cerr << "============== " << interpreter.getQueryTree()->dumpTree() << std::endl; - set_key = PreparedSetKey::forSubquery(query_tree->getTreeHash()); + set_key = query_tree->getTreeHash(); } else - set_key = PreparedSetKey::forSubquery(right_in_operand->getTreeHash()); + set_key = right_in_operand->getTreeHash(); // std::cerr << set_key.toString() << std::endl; // std::cerr << data.prepared_sets->getSets().size() << std::endl; @@ -1416,7 +1415,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool // for (const auto & [k, v] : data.prepared_sets->getSets()) // std::cerr << "... " << k.toString(); - if (auto set = data.prepared_sets->getFuture(set_key)) + if (auto set = data.prepared_sets->findSubquery(set_key)) return set; FutureSetPtr external_table_set; @@ -1449,7 +1448,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool // String set_id = right_in_operand->getColumnName(); //bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; SubqueryForSet subquery_for_set; // = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); - subquery_for_set.key = set_key.toString(); //right_in_operand->getColumnName(); + subquery_for_set.key = PreparedSets::toString(set_key, {}); //right_in_operand->getColumnName(); /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 678258489f4..5b8f0c60d35 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -189,11 +189,11 @@ public: // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); auto key = subquery_or_table_name->getColumnName(); - auto set_key = PreparedSetKey::forSubquery(database_and_table_name->getTreeHash()); + auto set_key = database_and_table_name->getTreeHash(); // std::cerr << "====== Adding key " << set_key.toString() << std::endl; - if (!prepared_sets->getFuture(set_key)) + if (!prepared_sets->findSubquery(set_key)) { SubqueryForSet subquery_for_set; subquery_for_set.key = std::move(key); diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 95cfa57857b..f74fe86acce 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -33,7 +33,7 @@ public: SetPtr get() const override { return set; } SetPtr buildOrderedSetInplace(const ContextPtr & context) override; - DataTypes getTypes() const override { return set->getElementsTypes(); } + const DataTypes & getTypes() const override { return set->getElementsTypes(); } private: SetPtr set; @@ -43,7 +43,7 @@ private: FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} SetPtr FutureSetFromStorage::get() const { return set; } -DataTypes FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } +const DataTypes & FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); } SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) { @@ -51,47 +51,47 @@ SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) } -PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) -{ - /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, - /// just converts LowCardinality to ordinary types. - for (auto & type : types_) - type = recursiveRemoveLowCardinality(type); +// PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) +// { +// /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, +// /// just converts LowCardinality to ordinary types. +// for (auto & type : types_) +// type = recursiveRemoveLowCardinality(type); - PreparedSetKey key; - key.ast_hash = hash; - key.types = std::move(types_); - return key; -} +// PreparedSetKey key; +// key.ast_hash = hash; +// key.types = std::move(types_); +// return key; +// } -PreparedSetKey PreparedSetKey::forSubquery(Hash hash) -{ - PreparedSetKey key; - key.ast_hash = hash; - return key; -} +// PreparedSetKey PreparedSetKey::forSubquery(Hash hash) +// { +// PreparedSetKey key; +// key.ast_hash = hash; +// return key; +// } -bool PreparedSetKey::operator==(const PreparedSetKey & other) const -{ - if (ast_hash != other.ast_hash) - return false; +// bool PreparedSetKey::operator==(const PreparedSetKey & other) const +// { +// if (ast_hash != other.ast_hash) +// return false; - if (types.size() != other.types.size()) - return false; +// if (types.size() != other.types.size()) +// return false; - for (size_t i = 0; i < types.size(); ++i) - { - if (!types[i]->equals(*other.types[i])) - return false; - } +// for (size_t i = 0; i < types.size(); ++i) +// { +// if (!types[i]->equals(*other.types[i])) +// return false; +// } - return true; -} +// return true; +// } -String PreparedSetKey::toString() const +String PreparedSets::toString(const PreparedSets::Hash & hash, const DataTypes & types) { WriteBufferFromOwnString buf; - buf << "__set_" << ast_hash.first << "_" << ast_hash.second; + buf << "__set_" << hash.first << "_" << hash.second; if (!types.empty()) { buf << "("; @@ -143,54 +143,104 @@ String PreparedSetKey::toString() const // it->second->addStorage(std::move(storage)); // } +static bool equals(const DataTypes & lhs, const DataTypes & rhs) +{ + size_t size = lhs.size(); + if (size != rhs.size()) + return false; -FutureSetPtr PreparedSets::addFromStorage(const PreparedSetKey & key, SetPtr set_) + for (size_t i = 0; i < size; ++i) + { + if (!lhs[i]->equals(*rhs[i])) + return false; + } + + return true; +} + +static bool tryInsertSet(std::vector> & sets, FutureSetPtr new_set) +{ + auto types = new_set->getTypes(); + for (const auto & set : sets) + if (equals(set->getTypes(), new_set->getTypes())) + return false; + + sets.push_back(std::move(new_set)); + return true; +} + +static FutureSetPtr findSet(const std::vector> & sets, const DataTypes & types) +{ + for (const auto & set : sets) + if (equals(set->getTypes(), types)) + return set; + + return nullptr; +} + +FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) { auto from_storage = std::make_shared(std::move(set_)); - auto [it, inserted] = sets.emplace(key, std::move(from_storage)); + auto & sets_by_hash = sets[key]; - if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + if (!tryInsertSet(sets_by_hash, from_storage)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_storage->getTypes())); - return it->second; + return from_storage; } -FutureSetPtr PreparedSets::addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings) +FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) { auto from_tuple = std::make_shared(std::move(block), settings); - auto [it, inserted] = sets.emplace(key, std::move(from_tuple)); + auto & sets_by_hash = sets[key]; - if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + if (!tryInsertSet(sets_by_hash, from_tuple)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_tuple->getTypes())); - return it->second; + return from_tuple; } -FutureSetPtr PreparedSets::addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) +FutureSetPtr PreparedSets::addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) { - auto id = subquery.key; - auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings.transform_null_in); - auto [it, inserted] = sets.emplace(key, from_subquery); + auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings); + auto [it, inserted] = sets_from_subqueries.emplace(key, from_subquery); if (!inserted) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", key.toString()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {})); // std::cerr << key.toString() << std::endl; // std::cerr << "========= PreparedSets::addFromSubquery\n"; // std::cerr << StackTrace().toString() << std::endl; - subqueries.emplace_back(SetAndName{.name = id, .set = std::move(from_subquery)}); - return it->second; + return from_subquery; } -FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const +FutureSetPtr PreparedSets::find(const Hash & key, const DataTypes & types) const { auto it = sets.find(key); if (it == sets.end()) - return {}; + return nullptr; + + return findSet(it->second, types); +} + +std::shared_ptr PreparedSets::findSubquery(const Hash & key) const +{ + auto it = sets_from_subqueries.find(key); + if (it == sets_from_subqueries.end()) + return nullptr; + return it->second; } +// FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const +// { +// auto it = sets.find(key); +// if (it == sets.end()) +// return {}; +// return it->second; +// } + // SetPtr PreparedSets::get(const PreparedSetKey & key) const // { // auto it = sets.find(key); @@ -210,10 +260,14 @@ FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // return res; // } -PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries() +std::vector> PreparedSets::detachSubqueries() { - auto res = std::move(subqueries); - subqueries = SubqueriesForSets(); + std::vector> res; + res.reserve(sets_from_subqueries.size()); + for (auto & [_, set] : sets_from_subqueries) + res.push_back(std::move(set)); + + sets_from_subqueries.clear(); return res; } @@ -267,10 +321,10 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) if (!context->getSettingsRef().use_index_for_in_with_subqueries) return nullptr; - if (subquery.set) + if (auto set = get()) { - if (subquery.set->hasExplicitSetElements()) - return subquery.set; + if (set->hasExplicitSetElements()) + return set; return nullptr; } @@ -278,10 +332,11 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) if (external_table_set) return subquery.set = external_table_set->buildOrderedSetInplace(context); - auto plan = buildPlan(context, true); + auto plan = buildPlan(context); if (!plan) return nullptr; + subquery.set->fillSetElements(); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); pipeline.complete(std::make_shared(Block())); @@ -304,7 +359,7 @@ SetPtr FutureSetFromSubquery::get() const std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) { - return buildPlan(context, false); + return buildPlan(context); } static SizeLimits getSizeLimitsForSet(const Settings & settings) @@ -312,15 +367,12 @@ static SizeLimits getSizeLimitsForSet(const Settings & settings) return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); } -std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context, bool create_ordered_set) +std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context) { - if (subquery.set) + if (subquery.set->isCreated()) return nullptr; const auto & settings = context->getSettingsRef(); - auto size_limits = getSizeLimitsForSet(settings); - - subquery.set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); auto plan = subquery.detachSource(); auto description = subquery.key; @@ -355,15 +407,25 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set->finishInsert(); } -FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_) - : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)), transform_null_in(transform_null_in_) {} - -DataTypes FutureSetFromSubquery::getTypes() const +FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, const Settings & settings) + : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)) { - if (subquery.set) - return subquery.set->getElementsTypes(); + bool create_ordered_set = false; + auto size_limits = getSizeLimitsForSet(settings); + subquery.set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); + if (subquery.source) + subquery.set->setHeader(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName()); +} - return Set::getElementTypes(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName(), transform_null_in); +void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source) +{ + subquery.source = std::move(source); + subquery.set->setHeader(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName()); +} + +const DataTypes & FutureSetFromSubquery::getTypes() const +{ + return subquery.set->getElementsTypes(); } SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 1e04bbbe3e8..67f8a65ca17 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -37,7 +37,7 @@ public: virtual SetPtr get() const = 0; virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; - virtual DataTypes getTypes() const = 0; + virtual const DataTypes & getTypes() const = 0; }; using FutureSetPtr = std::shared_ptr; @@ -50,7 +50,7 @@ public: SetPtr get() const override; SetPtr buildOrderedSetInplace(const ContextPtr &) override; - DataTypes getTypes() const override; + const DataTypes & getTypes() const override; private: SetPtr set; @@ -86,7 +86,7 @@ public: class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this { public: - FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, bool transform_null_in_); + FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, const Settings & settings); SetPtr get() const override; @@ -94,74 +94,87 @@ public: std::unique_ptr build(const ContextPtr & context); - DataTypes getTypes() const override; + const DataTypes & getTypes() const override; SubqueryForSet & getSubquery() { return subquery; } + void setQueryPlan(std::unique_ptr source); private: SubqueryForSet subquery; FutureSetPtr external_table_set; - bool transform_null_in; - std::unique_ptr buildPlan(const ContextPtr & context, bool create_ordered_set); + std::unique_ptr buildPlan(const ContextPtr & context); }; -struct PreparedSetKey -{ - using Hash = std::pair; +// struct PreparedSetKey +// { +// using Hash = std::pair; - /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired - /// data types of set elements (two different Sets can be required for two tuples with the same contents - /// if left hand sides of the IN operators have different types). - static PreparedSetKey forLiteral(Hash hash, DataTypes types_); +// /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired +// /// data types of set elements (two different Sets can be required for two tuples with the same contents +// /// if left hand sides of the IN operators have different types). +// static PreparedSetKey forLiteral(Hash hash, DataTypes types_); - /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting - /// set is fully determined by the subquery. - static PreparedSetKey forSubquery(Hash hash); +// /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting +// /// set is fully determined by the subquery. +// static PreparedSetKey forSubquery(Hash hash); - Hash ast_hash; - DataTypes types; /// Empty for subqueries. +// Hash ast_hash; +// DataTypes types; /// Empty for subqueries. - bool operator==(const PreparedSetKey & other) const; +// bool operator==(const PreparedSetKey & other) const; - String toString() const; +// String toString() const; - struct Hashing - { - UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } - }; -}; +// struct Hashing +// { +// UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } +// }; +// }; class PreparedSets { public: - struct SetAndName + + using Hash = std::pair; + struct Hashing { - String name; - std::shared_ptr set; + UInt64 operator()(const Hash & key) const { return key.first ^ key.second; } }; - using SubqueriesForSets = std::vector; - FutureSetPtr addFromStorage(const PreparedSetKey & key, SetPtr set_); - FutureSetPtr addFromTuple(const PreparedSetKey & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery(const PreparedSetKey & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); + // struct SetAndName + // { + // String name; + // std::shared_ptr set; + // }; - FutureSetPtr getFuture(const PreparedSetKey & key) const; + using Sets = std::unordered_map>, Hashing>; + using SetsFromSubqueries = std::unordered_map, Hashing>; + + FutureSetPtr addFromStorage(const Hash & key, SetPtr set_); + FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings); + FutureSetPtr addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); + + FutureSetPtr find(const Hash & key, const DataTypes & types) const; + std::shared_ptr findSubquery(const Hash & key) const; + + //FutureSetPtr getFuture(const PreparedSetKey & key) const; /// Get subqueries and clear them. /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. /// SetPtr would still be available for consumers of PreparedSets. - SubqueriesForSets detachSubqueries(); + std::vector> detachSubqueries(); - const std::unordered_map & getSets() const { return sets; } + const Sets & getNormalSets() const { return sets; } + const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; } bool empty() const; -private: - std::unordered_map sets; + static String toString(const Hash & key, const DataTypes & types); - /// This is the information required for building sets - SubqueriesForSets subqueries; +private: + Sets sets; + SetsFromSubqueries sets_from_subqueries; }; using PreparedSetsPtr = std::shared_ptr; diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index a6908d2bdd3..df30992dd35 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -47,6 +47,8 @@ public: { } + void fillSetElements() { fill_set_elements = true; } + /** Set can be created either from AST or from a stream of data (subquery result). */ diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 84ab453c065..3b9512436ad 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -61,9 +61,8 @@ public: if (storage_set) { /// Handle storage_set as ready set. - auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); + auto set_key = in_second_argument->getTreeHash(); sets.addFromStorage(set_key, storage_set->getSet()); - //planner_context.registerSet(set_key, PlannerSet(FutureSet(storage_set->getSet()))); } else if (const auto * constant_node = in_second_argument->as()) { @@ -82,8 +81,8 @@ public: if (const auto * low_cardinality_type = typeid_cast(element_type.get())) element_type = low_cardinality_type->getDictionaryType(); - auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); - if (sets.getFuture(set_key)) + auto set_key = in_second_argument->getTreeHash(); + if (sets.find(set_key, set_element_types)) return; sets.addFromTuple(set_key, std::move(set), settings); @@ -94,8 +93,8 @@ public: in_second_argument_node_type == QueryTreeNodeType::UNION || in_second_argument_node_type == QueryTreeNodeType::TABLE) { - auto set_key = PreparedSetKey::forSubquery(in_second_argument->getTreeHash()); - if (sets.getFuture(set_key)) + auto set_key = in_second_argument->getTreeHash(); + if (sets.findSubquery(set_key)) return; auto subquery_to_execute = in_second_argument; diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 4ac81e28f92..e7c10f6ef7a 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -933,18 +933,18 @@ void addBuildSubqueriesForSetsStepIfNeeded( auto subqueries = planner_context->getPreparedSets().detachSubqueries(); std::unordered_set useful_sets; - PreparedSets::SubqueriesForSets subqueries_for_sets; + //PreparedSets::SubqueriesForSets subqueries_for_sets; for (const auto & actions_to_execute : result_actions_to_execute) collectSetsFromActionsDAG(actions_to_execute, useful_sets); - auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.set.get()); }; + auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); }; auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate)); subqueries.erase(it, subqueries.end()); for (auto & subquery : subqueries) { - auto & subquery_for_set = subquery.set->getSubquery(); + auto & subquery_for_set = subquery->getSubquery(); auto subquery_options = select_query_options.subquery(); Planner subquery_planner( subquery_for_set.query_tree, @@ -952,7 +952,7 @@ void addBuildSubqueriesForSetsStepIfNeeded( planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); - subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); + subquery->setQueryPlan(std::make_unique(std::move(subquery_planner).extractQueryPlan())); } //addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 49312e5c4ad..0c84052578c 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -651,14 +651,13 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma element_type = low_cardinality_type->getDictionaryType(); } - auto set_key = PreparedSetKey::forLiteral(in_second_argument->getTreeHash(), set_element_types); + auto set_key = in_second_argument->getTreeHash(); + auto set = planner_context->getPreparedSets().find(set_key, set_element_types); - - auto set = planner_context->getPreparedSets().getFuture(set_key); if (!set) throw Exception(ErrorCodes::LOGICAL_ERROR, "No set is registered for key {}", - set_key.toString()); + PreparedSets::toString(set_key, set_element_types)); ColumnWithTypeAndName column; column.name = planner_context->createSetKey(in_second_argument); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 751a1f95b7b..ab028d967d8 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -126,7 +126,7 @@ void CreatingSetsStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } -void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context) +void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subqueries, ContextPtr context) { DataStreams input_streams; input_streams.emplace_back(query_plan.getCurrentDataStream()); @@ -135,7 +135,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets plans.emplace_back(std::make_unique(std::move(query_plan))); query_plan = QueryPlan(); - for (auto & [description, future_set] : subqueries_for_sets) + for (auto & future_set : sets_from_subqueries) { if (future_set->get()) continue; @@ -170,7 +170,7 @@ std::vector> DelayedCreatingSetsStep::makePlansForSet // plans.emplace_back(std::make_unique(std::move(query_plan))); // query_plan = QueryPlan(); - for (auto & [description, future_set] : step.subqueries_for_sets) + for (auto & future_set : step.sets_from_subquery) { if (future_set->get()) continue; @@ -197,8 +197,8 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, } DelayedCreatingSetsStep::DelayedCreatingSetsStep( - DataStream input_stream, PreparedSets::SubqueriesForSets subqueries_for_sets_, ContextPtr context_) - : subqueries_for_sets(std::move(subqueries_for_sets_)), context(std::move(context_)) + DataStream input_stream, std::vector> sets_from_subquery_, ContextPtr context_) + : sets_from_subquery(std::move(sets_from_subquery_)), context(std::move(context_)) { input_streams = {input_stream}; output_stream = std::move(input_stream); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 2a6cb43c45a..c41bb782e0b 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -52,7 +52,7 @@ public: class DelayedCreatingSetsStep final : public IQueryPlanStep { public: - DelayedCreatingSetsStep(DataStream input_stream, PreparedSets::SubqueriesForSets subqueries_for_sets_, ContextPtr context_); + DelayedCreatingSetsStep(DataStream input_stream, std::vector> sets_from_subquery_, ContextPtr context_); String getName() const override { return "DelayedCreatingSets"; } @@ -61,14 +61,14 @@ public: static std::vector> makePlansForSets(DelayedCreatingSetsStep && step); ContextPtr getContext() const { return context; } - PreparedSets::SubqueriesForSets detachSubqueries() { return std::move(subqueries_for_sets); } + std::vector> detachSets() { return std::move(sets_from_subquery); } private: - PreparedSets::SubqueriesForSets subqueries_for_sets; + std::vector> sets_from_subquery; ContextPtr context; }; -void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context); +void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subquery, ContextPtr context); void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 63ba3d5b56c..5f04bf9ec18 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -273,7 +273,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes /// CreatingSets does not change header. /// We can push down filter and update header. /// Filter - DelayedCreatingSets - Something - child = std::make_unique(filter->getOutputStream(), delayed->detachSubqueries(), delayed->getContext()); + child = std::make_unique(filter->getOutputStream(), delayed->detachSets(), delayed->getContext()); std::swap(parent, child); std::swap(parent_node->children, child_node->children); std::swap(parent_node->children.front(), child_node->children.front()); diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index 3139fa5ed98..aa374446f3e 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -135,11 +135,11 @@ void CreatingSetsTransform::init() { is_initialized = true; - if (subquery.set) - { - //std::cerr << "=========== " << getInputPort().getHeader().dumpStructure() << std::endl; - subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); - } + // if (subquery.set) + // { + // //std::cerr << "=========== " << getInputPort().getHeader().dumpStructure() << std::endl; + // subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); + // } watch.restart(); startSubquery(); diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index f4a4b6e9af1..d205086639d 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -66,13 +66,14 @@ bool traverseASTFilter( return false; value = args.children.at(1); - PreparedSetKey set_key; - if ((value->as() || value->as())) - set_key = PreparedSetKey::forSubquery(value->getTreeHash()); - else - set_key = PreparedSetKey::forLiteral(value->getTreeHash(), {primary_key_type}); + PreparedSets::Hash set_key = value->getTreeHash(); + FutureSetPtr future_set; + + if ((value->as() || value->as())) + future_set = prepared_sets->findSubquery(set_key); + else + future_set = prepared_sets->find(set_key, {primary_key_type}); - FutureSetPtr future_set = prepared_sets->getFuture(set_key); if (!future_set) return false; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index ad3fb984ce7..1bdbbfb3d40 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -299,14 +299,13 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const if (ast_node && prepared_sets) { - auto hash = ast_node->getTreeHash(); - auto key = PreparedSetKey::forSubquery(hash); + auto key = ast_node->getTreeHash(); + const auto & sets = prepared_sets->getNormalSets(); + auto it = sets.find(key); + if (it != sets.end() && !it->second.empty()) + return it->second.at(0); - for (const auto & [k, v] : prepared_sets->getSets()) - { - if (k.ast_hash == hash) - return v; - } + return prepared_sets->findSubquery(key); } else if (dag_node) { @@ -324,9 +323,9 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); + return prepared_sets->findSubquery(ast_node->getTreeHash()); - return prepared_sets->getFuture(PreparedSetKey::forLiteral(ast_node->getTreeHash(), data_types)); + return prepared_sets->find(ast_node->getTreeHash(), data_types); } else if (dag_node) { @@ -369,14 +368,17 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( if (prepared_sets && ast_node) { if (ast_node->as() || ast_node->as()) - return prepared_sets->getFuture(PreparedSetKey::forSubquery(ast_node->getTreeHash())); + return prepared_sets->findSubquery(ast_node->getTreeHash()); auto tree_hash = ast_node->getTreeHash(); - for (const auto & [key, future_set] : prepared_sets->getSets()) - { - if (key.ast_hash == tree_hash && types_match(key.types)) + const auto & sets = prepared_sets->getNormalSets(); + auto it = sets.find(tree_hash); + if (it == sets.end()) + return nullptr; + + for (const auto & future_set : it->second) + if (types_match(future_set->getTypes())) return future_set; - } } else { From b4273a1963dd5f66858d4a7c0feb7bc85dedc184 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 14:53:13 +0000 Subject: [PATCH 068/171] Fixing some tests. --- src/Interpreters/ActionsVisitor.cpp | 11 ++++++++--- src/Interpreters/PreparedSets.cpp | 12 +++++++++--- src/Interpreters/PreparedSets.h | 2 -- src/Interpreters/Set.cpp | 16 ++++++---------- src/Interpreters/Set.h | 2 +- src/Planner/CollectSets.cpp | 8 +++++--- src/Planner/PlannerActionsVisitor.cpp | 17 +++++++++++------ src/Processors/QueryPlan/CreatingSetsStep.cpp | 8 ++++++-- 8 files changed, 46 insertions(+), 30 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 91c490f8a4c..8ddc035417b 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -445,9 +445,14 @@ FutureSetPtr makeExplicitSet( if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); + set_element_types = Set::getElementTypes(std::move(set_element_types), context->getSettingsRef().transform_null_in); + + // for (auto & element_type : set_element_types) + // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + // element_type = low_cardinality_type->getDictionaryType(); + + // if (!set_element_types.empty()) + // std::cerr << "========== " << set_element_types[0]->getName() << std::endl; auto set_key = right_arg->getTreeHash(); if (auto set = prepared_sets.find(set_key, set_element_types)) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index f74fe86acce..3281e7e9bb8 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -271,8 +271,6 @@ std::vector> PreparedSets::detachSubqueri return res; } -bool PreparedSets::empty() const { return sets.empty(); } - void SubqueryForSet::createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_) { source = std::make_unique(); @@ -330,13 +328,18 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) } if (external_table_set) - return subquery.set = external_table_set->buildOrderedSetInplace(context); + { + auto set = external_table_set->buildOrderedSetInplace(context); + if (set) + return subquery.set = set; + } auto plan = buildPlan(context); if (!plan) return nullptr; subquery.set->fillSetElements(); + subquery.set->initSetElements(); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); pipeline.complete(std::make_shared(Block())); @@ -377,6 +380,9 @@ std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & c auto plan = subquery.detachSource(); auto description = subquery.key; + if (!plan) + return nullptr; + auto creating_set = std::make_unique( plan->getCurrentDataStream(), description, diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 67f8a65ca17..967cef615c5 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -168,8 +168,6 @@ public: const Sets & getNormalSets() const { return sets; } const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; } - bool empty() const; - static String toString(const Hash & key, const DataTypes & types); private: diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 83005f3d3a7..17df5064642 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -103,22 +103,18 @@ void NO_INLINE Set::insertFromBlockImplCase( } -DataTypes Set::getElementTypes(const ColumnsWithTypeAndName & header, bool transform_null_in) +DataTypes Set::getElementTypes(DataTypes types, bool transform_null_in) { - DataTypes data_types; - data_types.reserve(header.size()); - - for (const auto & column : header) + for (auto & type : types) { - data_types.push_back(column.type); - if (const auto * low_cardinality_type = typeid_cast(data_types.back().get())) - data_types.back() = low_cardinality_type->getDictionaryType(); + if (const auto * low_cardinality_type = typeid_cast(type.get())) + type = low_cardinality_type->getDictionaryType(); if (!transform_null_in) - data_types.back() = removeNullable(data_types.back()); + type = removeNullable(type); } - return data_types; + return types; } diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index df30992dd35..d1218830969 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -92,7 +92,7 @@ public: bool areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const; void checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const; - static DataTypes getElementTypes(const ColumnsWithTypeAndName & header, bool transform_null_in); + static DataTypes getElementTypes(DataTypes types, bool transform_null_in); private: size_t keys_size = 0; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 3b9512436ad..43431bd46ae 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -77,9 +77,11 @@ public: if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); + set_element_types = Set::getElementTypes(std::move(set_element_types), settings.transform_null_in); + + // for (auto & element_type : set_element_types) + // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + // element_type = low_cardinality_type->getDictionaryType(); auto set_key = in_second_argument->getTreeHash(); if (sets.find(set_key, set_element_types)) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 0c84052578c..a7a2c6edc6c 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -639,6 +639,9 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma in_second_argument_node_type == QueryTreeNodeType::UNION || in_second_argument_node_type == QueryTreeNodeType::TABLE; + FutureSetPtr set; + auto set_key = in_second_argument->getTreeHash(); + if (!subquery_or_table) { set_element_types = {in_first_argument->getResultType()}; @@ -646,13 +649,15 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); - } + // for (auto & element_type : set_element_types) + // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + // element_type = low_cardinality_type->getDictionaryType(); - auto set_key = in_second_argument->getTreeHash(); - auto set = planner_context->getPreparedSets().find(set_key, set_element_types); + set_element_types = Set::getElementTypes(std::move(set_element_types), planner_context->getQueryContext()->getSettingsRef().transform_null_in); + set = planner_context->getPreparedSets().find(set_key, set_element_types); + } + else + set = planner_context->getPreparedSets().findSubquery(set_key); if (!set) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index ab028d967d8..54d81dd7650 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -190,10 +190,14 @@ std::vector> DelayedCreatingSetsStep::makePlansForSet void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context) { - if (!prepared_sets || prepared_sets->empty()) + if (!prepared_sets) return; - addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(), context); + auto subqueries = prepared_sets->detachSubqueries(); + if (subqueries.empty()) + return; + + addCreatingSetsStep(query_plan, std::move(subqueries), context); } DelayedCreatingSetsStep::DelayedCreatingSetsStep( From 518ba6cbf6ee96deec772eebfaf8417347215460 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 16:43:22 +0000 Subject: [PATCH 069/171] Fix more tests. --- src/Interpreters/PreparedSets.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 3281e7e9bb8..17918a46954 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -265,9 +265,8 @@ std::vector> PreparedSets::detachSubqueri std::vector> res; res.reserve(sets_from_subqueries.size()); for (auto & [_, set] : sets_from_subqueries) - res.push_back(std::move(set)); + res.push_back(set); - sets_from_subqueries.clear(); return res; } From 5a1f819e7b068f9416e0aa9db6aec8697baa2fb8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 17:33:09 +0000 Subject: [PATCH 070/171] Fix more tests. --- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Interpreters/PreparedSets.cpp | 39 ++++++++++++++++----------- src/Interpreters/PreparedSets.h | 12 ++++++--- src/Planner/CollectSets.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 6 ++++- src/Storages/KVStorageUtils.cpp | 2 +- src/Storages/MergeTree/RPNBuilder.cpp | 6 ++--- 7 files changed, 43 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 8ddc035417b..a9c0bbbbbc0 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -455,7 +455,7 @@ FutureSetPtr makeExplicitSet( // std::cerr << "========== " << set_element_types[0]->getName() << std::endl; auto set_key = right_arg->getTreeHash(); - if (auto set = prepared_sets.find(set_key, set_element_types)) + if (auto set = prepared_sets.findTuple(set_key, set_element_types)) return set; /// Already prepared. Block block; diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 17918a46954..144436589a0 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -178,21 +178,10 @@ static FutureSetPtr findSet(const std::vector> & sets return nullptr; } -FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) -{ - auto from_storage = std::make_shared(std::move(set_)); - auto & sets_by_hash = sets[key]; - - if (!tryInsertSet(sets_by_hash, from_storage)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_storage->getTypes())); - - return from_storage; -} - FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) { auto from_tuple = std::make_shared(std::move(block), settings); - auto & sets_by_hash = sets[key]; + auto & sets_by_hash = sets_from_tuple[key]; if (!tryInsertSet(sets_by_hash, from_tuple)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_tuple->getTypes())); @@ -200,6 +189,17 @@ FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Set return from_tuple; } +FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) +{ + auto from_storage = std::make_shared(std::move(set_)); + auto [it, inserted] = sets_from_storage.emplace(key, from_storage); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {})); + + return from_storage; +} + FutureSetPtr PreparedSets::addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) { auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings); @@ -215,10 +215,10 @@ FutureSetPtr PreparedSets::addFromSubquery(const Hash & key, SubqueryForSet subq return from_subquery; } -FutureSetPtr PreparedSets::find(const Hash & key, const DataTypes & types) const +FutureSetPtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const { - auto it = sets.find(key); - if (it == sets.end()) + auto it = sets_from_tuple.find(key); + if (it == sets_from_tuple.end()) return nullptr; return findSet(it->second, types); @@ -233,6 +233,15 @@ std::shared_ptr PreparedSets::findSubquery(const Hash & k return it->second; } +std::shared_ptr PreparedSets::findStorage(const Hash & key) const +{ + auto it = sets_from_storage.find(key); + if (it == sets_from_storage.end()) + return nullptr; + + return it->second; +} + // FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const // { // auto it = sets.find(key); diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 967cef615c5..49545ad1e2a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -148,14 +148,16 @@ public: // std::shared_ptr set; // }; - using Sets = std::unordered_map>, Hashing>; + using SetsFromTuple = std::unordered_map>, Hashing>; + using SetsFromStorage = std::unordered_map, Hashing>; using SetsFromSubqueries = std::unordered_map, Hashing>; FutureSetPtr addFromStorage(const Hash & key, SetPtr set_); FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings); FutureSetPtr addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); - FutureSetPtr find(const Hash & key, const DataTypes & types) const; + FutureSetPtr findTuple(const Hash & key, const DataTypes & types) const; + std::shared_ptr findStorage(const Hash & key) const; std::shared_ptr findSubquery(const Hash & key) const; //FutureSetPtr getFuture(const PreparedSetKey & key) const; @@ -165,13 +167,15 @@ public: /// SetPtr would still be available for consumers of PreparedSets. std::vector> detachSubqueries(); - const Sets & getNormalSets() const { return sets; } + const SetsFromTuple & getSetsFromTuple() const { return sets_from_tuple; } + const SetsFromStorage & getSetsFromStorage() const { return sets_from_storage; } const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; } static String toString(const Hash & key, const DataTypes & types); private: - Sets sets; + SetsFromTuple sets_from_tuple; + SetsFromStorage sets_from_storage; SetsFromSubqueries sets_from_subqueries; }; diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index 43431bd46ae..b0e6b272c9f 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -84,7 +84,7 @@ public: // element_type = low_cardinality_type->getDictionaryType(); auto set_key = in_second_argument->getTreeHash(); - if (sets.find(set_key, set_element_types)) + if (sets.findTuple(set_key, set_element_types)) return; sets.addFromTuple(set_key, std::move(set), settings); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index a7a2c6edc6c..40b6ec109dc 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -654,10 +654,14 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma // element_type = low_cardinality_type->getDictionaryType(); set_element_types = Set::getElementTypes(std::move(set_element_types), planner_context->getQueryContext()->getSettingsRef().transform_null_in); - set = planner_context->getPreparedSets().find(set_key, set_element_types); + set = planner_context->getPreparedSets().findTuple(set_key, set_element_types); } else + { set = planner_context->getPreparedSets().findSubquery(set_key); + if (!set) + set = planner_context->getPreparedSets().findStorage(set_key); + } if (!set) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index d205086639d..b3b60fc064c 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -72,7 +72,7 @@ bool traverseASTFilter( if ((value->as() || value->as())) future_set = prepared_sets->findSubquery(set_key); else - future_set = prepared_sets->find(set_key, {primary_key_type}); + future_set = prepared_sets->findTuple(set_key, {primary_key_type}); if (!future_set) return false; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 1bdbbfb3d40..0b42111be45 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -300,7 +300,7 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const if (ast_node && prepared_sets) { auto key = ast_node->getTreeHash(); - const auto & sets = prepared_sets->getNormalSets(); + const auto & sets = prepared_sets->getSetsFromTuple(); auto it = sets.find(key); if (it != sets.end() && !it->second.empty()) return it->second.at(0); @@ -325,7 +325,7 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) if (ast_node->as() || ast_node->as()) return prepared_sets->findSubquery(ast_node->getTreeHash()); - return prepared_sets->find(ast_node->getTreeHash(), data_types); + return prepared_sets->findTuple(ast_node->getTreeHash(), data_types); } else if (dag_node) { @@ -371,7 +371,7 @@ FutureSetPtr RPNBuilderTreeNode::tryGetPreparedSet( return prepared_sets->findSubquery(ast_node->getTreeHash()); auto tree_hash = ast_node->getTreeHash(); - const auto & sets = prepared_sets->getNormalSets(); + const auto & sets = prepared_sets->getSetsFromTuple(); auto it = sets.find(tree_hash); if (it == sets.end()) return nullptr; From c0b1d7f4f2bc9dc92c35b751c37a5f747f0a297a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 19:12:22 +0000 Subject: [PATCH 071/171] Fixing 01774_tuple_null_in --- src/Interpreters/ActionsVisitor.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index a9c0bbbbbc0..9125b38b48c 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -445,17 +445,17 @@ FutureSetPtr makeExplicitSet( if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - set_element_types = Set::getElementTypes(std::move(set_element_types), context->getSettingsRef().transform_null_in); - - // for (auto & element_type : set_element_types) - // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - // element_type = low_cardinality_type->getDictionaryType(); + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); // if (!set_element_types.empty()) // std::cerr << "========== " << set_element_types[0]->getName() << std::endl; + auto set_element_keys = Set::getElementTypes(set_element_types, context->getSettingsRef().transform_null_in); + auto set_key = right_arg->getTreeHash(); - if (auto set = prepared_sets.findTuple(set_key, set_element_types)) + if (auto set = prepared_sets.findTuple(set_key, set_element_keys)) return set; /// Already prepared. Block block; From e4f057e3040d6088d5fffec568fc167fefba56b8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 19:40:37 +0000 Subject: [PATCH 072/171] Fix Set index with constant nullable comparison. --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +- ...2789_set_index_nullable_condition_bug.reference | 1 + .../02789_set_index_nullable_condition_bug.sql | 14 ++++++++++++++ 3 files changed, 16 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02789_set_index_nullable_condition_bug.reference create mode 100644 tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 120b3e43472..78e8b460433 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -554,7 +554,7 @@ void MergeTreeIndexConditionSet::traverseAST(ASTPtr & node) const if (atomFromAST(node)) { if (node->as() || node->as()) - node = makeASTFunction("__bitWrapperFunc", node); + else } else node = std::make_shared(UNKNOWN_FIELD); diff --git a/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.reference b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.reference new file mode 100644 index 00000000000..e6ce356cd45 --- /dev/null +++ b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.reference @@ -0,0 +1 @@ +v1 v2 diff --git a/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql new file mode 100644 index 00000000000..84c53d35d71 --- /dev/null +++ b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql @@ -0,0 +1,14 @@ +CREATE OR REPLACE TABLE test_table +( + col1 String, + col2 String, + INDEX test_table_col2_idx col2 TYPE set(0) GRANULARITY 1 +) ENGINE = MergeTree() + ORDER BY col1 +AS SELECT 'v1', 'v2'; + +SELECT * FROM test_table +WHERE 1 == 1 AND col1 == col1 OR + 0 AND col2 == NULL; + +drop table if exists test_table; From 5bda0b535d9b161acf17980ebddd252e1dd4f587 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 19:45:03 +0000 Subject: [PATCH 073/171] Fixing build. --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 78e8b460433..6f4c5e48ea8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -1,3 +1,4 @@ +#include "Parsers/queryToString.h" #include #include @@ -554,7 +555,7 @@ void MergeTreeIndexConditionSet::traverseAST(ASTPtr & node) const if (atomFromAST(node)) { if (node->as() || node->as()) - else + node = makeASTFunction("__bitWrapperFunc", makeASTFunction("toUInt8OrDefault", node)); } else node = std::make_shared(UNKNOWN_FIELD); From 266e81702a3c7cf68ac3f78995ce7b1fea8686e3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Jun 2023 19:48:39 +0000 Subject: [PATCH 074/171] Fixing clang-tidy --- src/Interpreters/PreparedSets.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 144436589a0..03c18730cc5 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -88,10 +88,10 @@ SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) // return true; // } -String PreparedSets::toString(const PreparedSets::Hash & hash, const DataTypes & types) +String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & types) { WriteBufferFromOwnString buf; - buf << "__set_" << hash.first << "_" << hash.second; + buf << "__set_" << key.first << "_" << key.second; if (!types.empty()) { buf << "("; From 44798971f9bd4f17e1c856b5345532cd6c96f2a7 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 20 Jun 2023 18:25:38 -0700 Subject: [PATCH 075/171] Fix a test, add APPEND to suggest. --- src/Client/Suggest.cpp | 2 +- tests/queries/0_stateless/02346_into_outfile_and_stdout.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/Suggest.cpp b/src/Client/Suggest.cpp index 4e38add0ef5..c9cc27b4337 100644 --- a/src/Client/Suggest.cpp +++ b/src/Client/Suggest.cpp @@ -43,7 +43,7 @@ Suggest::Suggest() "IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE", "PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED", - "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "CLEANUP" + "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "CLEANUP", "APPEND" }); } diff --git a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh index 8ec086c97ef..38145ab3e51 100755 --- a/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh +++ b/tests/queries/0_stateless/02346_into_outfile_and_stdout.sh @@ -66,7 +66,7 @@ performBadQuery "bad_query_incorrect_usage" "SELECT 1, 2, 3 INTO OUTFILE AND STD performBadQuery "bad_query_no_into_outfile" "SELECT 1, 2, 3 AND STDOUT'" "SYNTAX_ERROR" -performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File ${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists exists, consider using APPEND or TRUNCATE if appropriate. (CANNOT_OPEN_FILE)" +performFileExists "bad_query_file_exists" "SELECT 1, 2, 3 INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out' AND STDOUT" "File ${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_bad_query_file_exists.out exists, consider using APPEND or TRUNCATE." performCompression "compression" "SELECT * FROM (SELECT 'Hello, World! From clickhouse.') INTO OUTFILE '${CLICKHOUSE_TMP}/test_into_outfile_and_stdout_compression.gz' AND STDOUT COMPRESSION 'GZ' FORMAT TabSeparated" From dab42af754735f6d3e002af224a8f2e708124e14 Mon Sep 17 00:00:00 2001 From: Manas Alekar Date: Tue, 20 Jun 2023 22:43:25 -0700 Subject: [PATCH 076/171] Fix another test. --- .../0_stateless/02050_clickhouse_client_local_exception.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh b/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh index eac085a914e..a2bb29377ac 100755 --- a/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh +++ b/tests/queries/0_stateless/02050_clickhouse_client_local_exception.sh @@ -10,4 +10,4 @@ function cleanup() rm "${CLICKHOUSE_TMP}/test_exception" } trap cleanup EXIT -$CLICKHOUSE_LOCAL --query="SELECT 1 INTO OUTFILE '${CLICKHOUSE_TMP}/test_exception' FORMAT Native" 2>&1 | grep -q "Code: 76. DB::ErrnoException:" && echo 'OK' || echo 'FAIL' ||: +$CLICKHOUSE_LOCAL --query="SELECT 1 INTO OUTFILE '${CLICKHOUSE_TMP}/test_exception' FORMAT Native" 2>&1 | grep -q "Code: 504. DB::Exception:" && echo 'OK' || echo 'FAIL' ||: From 5e4805665974ded066713ccdaacc76cfff618b5d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Jun 2023 09:21:01 +0000 Subject: [PATCH 077/171] Disable analyzer --- .../0_stateless/02494_query_cache_nested_query_bug.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index f451e9b0709..c5ca1d88b4d 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest +# Tags: no-parallel # Tag no-parallel: Messes with internal cache CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -16,7 +16,7 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (3) (4) (5)" # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need need to check the logs. -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1" 2>&1 | grep "Aggregated. " | wc -l -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0" 2>&1 | grep "Aggregated. " | wc -l ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" From 0149f905decbb5de715245463c82f470bd8af7c1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Jun 2023 11:26:54 +0200 Subject: [PATCH 078/171] Update MergeTreeIndexSet.cpp --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 6f4c5e48ea8..7f6edfc7ec4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -1,4 +1,3 @@ -#include "Parsers/queryToString.h" #include #include From 446320b182e7ceba9df32307e7c18be9cbbcfe34 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Jun 2023 11:28:15 +0200 Subject: [PATCH 079/171] Update 02789_set_index_nullable_condition_bug.sql --- .../0_stateless/02789_set_index_nullable_condition_bug.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql index 84c53d35d71..fed030f170d 100644 --- a/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql +++ b/tests/queries/0_stateless/02789_set_index_nullable_condition_bug.sql @@ -1,4 +1,5 @@ -CREATE OR REPLACE TABLE test_table +drop table if exists test_table; +CREATE TABLE test_table ( col1 String, col2 String, From 75b7121759ac4bb1fc12078042d52a445ae319a5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 21 Jun 2023 12:37:06 +0000 Subject: [PATCH 080/171] Exclude fasttest again :-( --- .../queries/0_stateless/02494_query_cache_nested_query_bug.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index c5ca1d88b4d..394367919e9 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash -# Tags: no-parallel +# Tags: no-parallel, no-fasttest # Tag no-parallel: Messes with internal cache +# no-fasttest: Produces wrong results in fasttest, unclear why, didn't reproduce locally. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 5229544b559366a9a10df0ecb17485a54eee51fb Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 21 Jun 2023 22:17:39 +0200 Subject: [PATCH 081/171] Added function getCurrrentBatchAndScheduleNext to IObjectStorageIteratorAsync --- src/Disks/ObjectStorages/ObjectStorageIterator.h | 6 ++++++ .../ObjectStorageIteratorAsync.cpp | 16 ++++++++++++++++ .../ObjectStorages/ObjectStorageIteratorAsync.h | 3 ++- src/Storages/StorageAzureBlob.cpp | 11 +++++++---- 4 files changed, 31 insertions(+), 5 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.h b/src/Disks/ObjectStorages/ObjectStorageIterator.h index 2ff5ce60acc..841b0ea6664 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIterator.h +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.h @@ -14,6 +14,7 @@ public: virtual bool isValid() = 0; virtual RelativePathWithMetadata current() = 0; virtual RelativePathsWithMetadata currentBatch() = 0; + virtual std::optional getCurrrentBatchAndScheduleNext() = 0; virtual size_t getAccumulatedSize() const = 0; virtual ~IObjectStorageIterator() = default; @@ -53,6 +54,11 @@ public: return batch; } + virtual std::optional getCurrrentBatchAndScheduleNext() override + { + return std::nullopt; + } + size_t getAccumulatedSize() const override { return batch.size(); diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index f91c19f2fb9..7425f629a5a 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -100,6 +100,22 @@ RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch() return current_batch; } +std::optional IObjectStorageIteratorAsync::getCurrrentBatchAndScheduleNext() +{ + std::lock_guard lock(mutex); + if (!is_initialized) + nextBatch(); + + if (current_batch_iterator != current_batch.end()) + { + auto temp_current_batch = current_batch; + nextBatch(); + return temp_current_batch; + } + + return std::nullopt; +} + size_t IObjectStorageIteratorAsync::getAccumulatedSize() const { return accumulated_size.load(std::memory_order_relaxed); diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h index a2b06da9a91..b0dd3cef39c 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -27,6 +27,7 @@ public: RelativePathWithMetadata current() override; RelativePathsWithMetadata currentBatch() override; size_t getAccumulatedSize() const override; + std::optional getCurrrentBatchAndScheduleNext() override; ~IObjectStorageIteratorAsync() override { @@ -48,7 +49,7 @@ protected: bool is_initialized{false}; bool is_finished{false}; - mutable std::mutex mutex; + mutable std::recursive_mutex mutex; ThreadPool list_objects_pool; ThreadPoolCallbackRunner list_objects_scheduler; std::future outcome_future; diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index b9d59f04001..91dc92f09e8 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -882,7 +882,6 @@ StorageAzureBlobSource::Iterator::Iterator( RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { - std::lock_guard lock(next_mutex); if (is_finished) return {}; @@ -901,6 +900,7 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() { bool need_new_batch = false; { + std::lock_guard lock(next_mutex); need_new_batch = !blobs_with_metadata || index >= blobs_with_metadata->size(); } @@ -909,10 +909,10 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() RelativePathsWithMetadata new_batch; while (new_batch.empty()) { - if (object_storage_iterator->isValid()) + auto result = object_storage_iterator->getCurrrentBatchAndScheduleNext(); + if (result.has_value()) { - new_batch = object_storage_iterator->currentBatch(); - object_storage_iterator->nextBatch(); + new_batch = result.value(); } else { @@ -945,6 +945,7 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); const auto & idxs = typeid_cast(*block.getByName("_idx").column); + std::lock_guard lock(next_mutex); blob_path_with_globs.reset(); blob_path_with_globs.emplace(); for (UInt64 idx : idxs.getData()) @@ -960,6 +961,7 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() if (outer_blobs) outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); + std::lock_guard lock(next_mutex); blobs_with_metadata = std::move(new_batch); for (const auto & [_, info] : *blobs_with_metadata) total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); @@ -967,6 +969,7 @@ RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() } size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + std::lock_guard lock(next_mutex); return (*blobs_with_metadata)[current_index]; } } From 0af028ab3f19dc1dc3501d5e5b2dd0e9aa540fdc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 21 Jun 2023 20:55:27 +0000 Subject: [PATCH 082/171] Refactor a bit more. --- src/Interpreters/ActionsVisitor.cpp | 12 +- src/Interpreters/GlobalSubqueriesVisitor.h | 9 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/PreparedSets.cpp | 137 +++++++++++------ src/Interpreters/PreparedSets.h | 145 +++++++++--------- src/Planner/CollectSets.cpp | 28 +--- src/Planner/Planner.cpp | 8 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 26 ++-- src/Processors/QueryPlan/CreatingSetsStep.h | 16 +- .../Transforms/CreatingSetsTransform.cpp | 60 +++----- .../Transforms/CreatingSetsTransform.h | 8 +- src/QueryPipeline/QueryPipelineBuilder.cpp | 8 +- src/QueryPipeline/QueryPipelineBuilder.h | 8 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- 14 files changed, 232 insertions(+), 239 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 9125b38b48c..7c3af4ce691 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1406,7 +1406,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery()); - auto query_tree = interpreter.getQueryTree(); + const auto & query_tree = interpreter.getQueryTree(); if (auto * query_node = query_tree->as()) query_node->setIsSubquery(true); set_key = query_tree->getTreeHash(); @@ -1449,11 +1449,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool } } - /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. - // String set_id = right_in_operand->getColumnName(); - //bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in; - SubqueryForSet subquery_for_set; // = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in); - subquery_for_set.key = PreparedSets::toString(set_key, {}); //right_in_operand->getColumnName(); + std::unique_ptr source = std::make_unique(); /** The following happens for GLOBAL INs or INs: * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, @@ -1465,10 +1461,10 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool */ { auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {}); - subquery_for_set.createSource(*interpreter); + interpreter->buildQueryPlan(*source); } - return data.prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), data.getContext()->getSettingsRef(), std::move(external_table_set)); + return data.prepared_sets->addFromSubquery(set_key, std::move(source), nullptr, std::move(external_table_set), data.getContext()->getSettingsRef()); } else { diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 5b8f0c60d35..8c784d3c2ff 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -188,20 +188,17 @@ public: { // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); // subquery_for_set.createSource(*interpreter, external_storage); - auto key = subquery_or_table_name->getColumnName(); auto set_key = database_and_table_name->getTreeHash(); // std::cerr << "====== Adding key " << set_key.toString() << std::endl; if (!prepared_sets->findSubquery(set_key)) { - SubqueryForSet subquery_for_set; - subquery_for_set.key = std::move(key); - subquery_for_set.table = std::move(external_storage); - subquery_for_set.createSource(*interpreter); + std::unique_ptr source = std::make_unique(); + interpreter->buildQueryPlan(*source); //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; - auto future_set = prepared_sets->addFromSubquery(set_key, std::move(subquery_for_set), getContext()->getSettingsRef(), nullptr); + auto future_set = prepared_sets->addFromSubquery(set_key, std::move(source), std::move(external_storage), nullptr, getContext()->getSettingsRef()); // std::cerr << "... Future set " << reinterpret_cast(external_storage_holder.get()) << " " << reinterpret_cast(future_set.get()) << std::endl; external_storage_holder->future_set = std::move(future_set); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 52238ad5932..01736731f2e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -3151,7 +3151,7 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan) { - auto subqueries = prepared_sets->detachSubqueries(); + auto subqueries = prepared_sets->getSubqueries(); if (!subqueries.empty()) { diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 03c18730cc5..2aa3e02c713 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -200,17 +200,42 @@ FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_) return from_storage; } -FutureSetPtr PreparedSets::addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set) +FutureSetPtr PreparedSets::addFromSubquery( + const Hash & key, + std::unique_ptr source, + StoragePtr external_table, + FutureSetPtr external_table_set, + const Settings & settings) { - auto from_subquery = std::make_shared(std::move(subquery), std::move(external_table_set), settings); + auto from_subquery = std::make_shared( + toString(key, {}), + std::move(source), + std::move(external_table), + std::move(external_table_set), + settings); + auto [it, inserted] = sets_from_subqueries.emplace(key, from_subquery); if (!inserted) throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {})); - // std::cerr << key.toString() << std::endl; - // std::cerr << "========= PreparedSets::addFromSubquery\n"; - // std::cerr << StackTrace().toString() << std::endl; + return from_subquery; +} + +FutureSetPtr PreparedSets::addFromSubquery( + const Hash & key, + QueryTreeNodePtr query_tree, + const Settings & settings) +{ + auto from_subquery = std::make_shared( + toString(key, {}), + std::move(query_tree), + settings); + + auto [it, inserted] = sets_from_subqueries.emplace(key, from_subquery); + + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {})); return from_subquery; } @@ -269,7 +294,7 @@ std::shared_ptr PreparedSets::findStorage(const Hash & key // return res; // } -std::vector> PreparedSets::detachSubqueries() +std::vector> PreparedSets::getSubqueries() { std::vector> res; res.reserve(sets_from_subqueries.size()); @@ -279,25 +304,25 @@ std::vector> PreparedSets::detachSubqueri return res; } -void SubqueryForSet::createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_) -{ - source = std::make_unique(); - interpreter.buildQueryPlan(*source); - if (table_) - table = table_; -} +// void SubqueryForSet::createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_) +// { +// source = std::make_unique(); +// interpreter.buildQueryPlan(*source); +// if (table_) +// table = table_; +// } -bool SubqueryForSet::hasSource() const -{ - return source != nullptr; -} +// bool SubqueryForSet::hasSource() const +// { +// return source != nullptr; +// } -QueryPlanPtr SubqueryForSet::detachSource() -{ - auto res = std::move(source); - source = nullptr; - return res; -} +// QueryPlanPtr SubqueryForSet::detachSource() +// { +// auto res = std::move(source); +// source = nullptr; +// return res; +// } std::variant, SharedSet> PreparedSetsCache::findOrPromiseToBuild(const String & key) @@ -339,15 +364,15 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) { auto set = external_table_set->buildOrderedSetInplace(context); if (set) - return subquery.set = set; + return set_and_key->set = set; } auto plan = buildPlan(context); if (!plan) return nullptr; - subquery.set->fillSetElements(); - subquery.set->initSetElements(); + set_and_key->set->fillSetElements(); + set_and_key->set->initSetElements(); auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); pipeline.complete(std::make_shared(Block())); @@ -355,15 +380,15 @@ SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) CompletedPipelineExecutor executor(pipeline); executor.execute(); - subquery.set->checkIsCreated(); + set_and_key->set->checkIsCreated(); - return subquery.set; + return set_and_key->set; } SetPtr FutureSetFromSubquery::get() const { - if (subquery.set != nullptr && subquery.set->isCreated()) - return subquery.set; + if (set_and_key->set != nullptr && set_and_key->set->isCreated()) + return set_and_key->set; return nullptr; } @@ -380,22 +405,20 @@ static SizeLimits getSizeLimitsForSet(const Settings & settings) std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context) { - if (subquery.set->isCreated()) + if (set_and_key->set->isCreated()) return nullptr; const auto & settings = context->getSettingsRef(); - auto plan = subquery.detachSource(); - auto description = subquery.key; + auto plan = std::move(source); if (!plan) return nullptr; auto creating_set = std::make_unique( plan->getCurrentDataStream(), - description, - subquery, - shared_from_this(), + set_and_key, + external_table, SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), context); creating_set->setStepDescription("Create set for subquery"); @@ -421,25 +444,49 @@ FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) set->finishInsert(); } -FutureSetFromSubquery::FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, const Settings & settings) - : subquery(std::move(subquery_)), external_table_set(std::move(external_table_set_)) +FutureSetFromSubquery::FutureSetFromSubquery( + String key, + std::unique_ptr source_, + StoragePtr external_table_, + FutureSetPtr external_table_set_, + const Settings & settings) + : 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); + bool create_ordered_set = false; auto size_limits = getSizeLimitsForSet(settings); - subquery.set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); - if (subquery.source) - subquery.set->setHeader(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName()); + set_and_key->set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); + set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); } -void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source) +FutureSetFromSubquery::FutureSetFromSubquery( + String key, + QueryTreeNodePtr query_tree_, + //FutureSetPtr external_table_set_, + const Settings & settings) + : query_tree(std::move(query_tree_)) { - subquery.source = std::move(source); - subquery.set->setHeader(subquery.source->getCurrentDataStream().header.getColumnsWithTypeAndName()); + set_and_key = std::make_shared(); + set_and_key->key = std::move(key); + + bool create_ordered_set = false; + auto size_limits = getSizeLimitsForSet(settings); + set_and_key->set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); +} + +void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source_) +{ + source = std::move(source_); + set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); } const DataTypes & FutureSetFromSubquery::getTypes() const { - return subquery.set->getElementsTypes(); + return set_and_key->set->getElementsTypes(); } SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 49545ad1e2a..339f4a6b435 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -24,6 +24,16 @@ using QueryTreeNodePtr = std::shared_ptr; struct Settings; +/// This is a structure for prepared sets cache. +/// SetPtr can be taken from cache, so we should pass holder for it. +struct SetAndKey +{ + String key; + SetPtr set; +}; + +using SetAndKeyPtr = std::shared_ptr; + /// Represents a set in a query that might be referenced at analysis time and built later during execution. /// Also it can represent a constant set that is ready to use. /// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there @@ -35,22 +45,20 @@ public: virtual ~FutureSet() = default; virtual SetPtr get() const = 0; - virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; - virtual const DataTypes & getTypes() const = 0; + virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; }; using FutureSetPtr = std::shared_ptr; - class FutureSetFromStorage : public FutureSet { public: FutureSetFromStorage(SetPtr set_); SetPtr get() const override; - SetPtr buildOrderedSetInplace(const ContextPtr &) override; const DataTypes & getTypes() const override; + SetPtr buildOrderedSetInplace(const ContextPtr &) override; private: SetPtr set; @@ -58,80 +66,67 @@ private: /// Information on how to build set for the [GLOBAL] IN section. -class SubqueryForSet +// class SubqueryForSet +// { +// public: + +// void createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_ = nullptr); + +// bool hasSource() const; + +// /// Returns query plan for the set's source +// /// and removes it from SubqueryForSet because we need to build it only once. +// std::unique_ptr detachSource(); + +// /// Build this set from the result of the subquery. +// String key; +// SetPtr set; + +// /// If set, put the result into the table. +// /// This is a temporary table for transferring to remote servers for distributed query processing. +// StoragePtr table; + +// /// The source is obtained using the InterpreterSelectQuery subquery. +// std::unique_ptr source; +// QueryTreeNodePtr query_tree; +// }; + +class FutureSetFromSubquery : public FutureSet { public: + FutureSetFromSubquery( + String key, + std::unique_ptr source_, + StoragePtr external_table_, + FutureSetPtr external_table_set_, + const Settings & settings); - void createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_ = nullptr); - - bool hasSource() const; - - /// Returns query plan for the set's source - /// and removes it from SubqueryForSet because we need to build it only once. - std::unique_ptr detachSource(); - - /// Build this set from the result of the subquery. - String key; - SetPtr set; - - /// If set, put the result into the table. - /// This is a temporary table for transferring to remote servers for distributed query processing. - StoragePtr table; - - /// The source is obtained using the InterpreterSelectQuery subquery. - std::unique_ptr source; - QueryTreeNodePtr query_tree; -}; - -class FutureSetFromSubquery : public FutureSet, public std::enable_shared_from_this -{ -public: - FutureSetFromSubquery(SubqueryForSet subquery_, FutureSetPtr external_table_set_, const Settings & settings); + FutureSetFromSubquery( + String key, + QueryTreeNodePtr query_tree_, + //FutureSetPtr external_table_set_, + const Settings & settings); SetPtr get() const override; - + const DataTypes & getTypes() const override; SetPtr buildOrderedSetInplace(const ContextPtr & context) override; std::unique_ptr build(const ContextPtr & context); - const DataTypes & getTypes() const override; - - SubqueryForSet & getSubquery() { return subquery; } - void setQueryPlan(std::unique_ptr source); + QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); } + void setQueryPlan(std::unique_ptr source_); private: - SubqueryForSet subquery; + SetAndKeyPtr set_and_key; + StoragePtr external_table; FutureSetPtr external_table_set; + std::unique_ptr source; + QueryTreeNodePtr query_tree; + std::unique_ptr buildPlan(const ContextPtr & context); }; -// struct PreparedSetKey -// { -// using Hash = std::pair; - -// /// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired -// /// data types of set elements (two different Sets can be required for two tuples with the same contents -// /// if left hand sides of the IN operators have different types). -// static PreparedSetKey forLiteral(Hash hash, DataTypes types_); - -// /// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting -// /// set is fully determined by the subquery. -// static PreparedSetKey forSubquery(Hash hash); - -// Hash ast_hash; -// DataTypes types; /// Empty for subqueries. - -// bool operator==(const PreparedSetKey & other) const; - -// String toString() const; - -// struct Hashing -// { -// UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; } -// }; -// }; - class PreparedSets { public: @@ -142,30 +137,30 @@ public: UInt64 operator()(const Hash & key) const { return key.first ^ key.second; } }; - // struct SetAndName - // { - // String name; - // std::shared_ptr set; - // }; - using SetsFromTuple = std::unordered_map>, Hashing>; using SetsFromStorage = std::unordered_map, Hashing>; using SetsFromSubqueries = std::unordered_map, Hashing>; FutureSetPtr addFromStorage(const Hash & key, SetPtr set_); FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings); - FutureSetPtr addFromSubquery(const Hash & key, SubqueryForSet subquery, const Settings & settings, FutureSetPtr external_table_set); + + FutureSetPtr addFromSubquery( + const Hash & key, + std::unique_ptr source, + StoragePtr external_table, + FutureSetPtr external_table_set, + const Settings & settings); + + FutureSetPtr addFromSubquery( + const Hash & key, + QueryTreeNodePtr query_tree, + const Settings & settings); FutureSetPtr findTuple(const Hash & key, const DataTypes & types) const; std::shared_ptr findStorage(const Hash & key) const; std::shared_ptr findSubquery(const Hash & key) const; - //FutureSetPtr getFuture(const PreparedSetKey & key) const; - - /// Get subqueries and clear them. - /// We need to build a plan for subqueries just once. That's why we can clear them after accessing them. - /// SetPtr would still be available for consumers of PreparedSets. - std::vector> detachSubqueries(); + std::vector> getSubqueries(); const SetsFromTuple & getSetsFromTuple() const { return sets_from_tuple; } const SetsFromStorage & getSetsFromStorage() const { return sets_from_storage; } diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index b0e6b272c9f..df3558745fb 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -46,12 +46,6 @@ public: auto in_second_argument_node_type = in_second_argument->getNodeType(); const auto & settings = planner_context.getQueryContext()->getSettingsRef(); - - // String set_key = planner_context.createSetKey(in_second_argument); - - // if (planner_context.hasSet(set_key)) - // return; - auto & sets = planner_context.getPreparedSets(); /// Tables and table functions are replaced with subquery at Analysis stage, except special Set table. @@ -129,27 +123,7 @@ public: subquery_to_execute = std::move(subquery_for_table); } - // auto subquery_options = select_query_options.subquery(); - // Planner subquery_planner( - // in_second_argument, - // subquery_options, - // planner_context.getGlobalPlannerContext()); - // subquery_planner.buildQueryPlanIfNeeded(); - - // const auto & settings = planner_context.getQueryContext()->getSettingsRef(); - // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - // bool tranform_null_in = settings.transform_null_in; - // auto set = std::make_shared(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in); - - SubqueryForSet subquery_for_set; - subquery_for_set.key = planner_context.createSetKey(in_second_argument); - subquery_for_set.query_tree = std::move(subquery_to_execute); - //subquery_for_set.source = std::make_unique(std::move(subquery_planner).extractQueryPlan()); - - /// TODO - sets.addFromSubquery(set_key, std::move(subquery_for_set), settings, nullptr); - - //planner_context.registerSet(set_key, PlannerSet(in_second_argument)); + sets.addFromSubquery(set_key, std::move(subquery_to_execute), settings); } else { diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index e7c10f6ef7a..d524f2bed98 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -930,7 +930,7 @@ void addBuildSubqueriesForSetsStepIfNeeded( const PlannerContextPtr & planner_context, const std::vector & result_actions_to_execute) { - auto subqueries = planner_context->getPreparedSets().detachSubqueries(); + auto subqueries = planner_context->getPreparedSets().getSubqueries(); std::unordered_set useful_sets; //PreparedSets::SubqueriesForSets subqueries_for_sets; @@ -944,10 +944,10 @@ void addBuildSubqueriesForSetsStepIfNeeded( for (auto & subquery : subqueries) { - auto & subquery_for_set = subquery->getSubquery(); + auto query_tree = subquery->detachQueryTree(); auto subquery_options = select_query_options.subquery(); Planner subquery_planner( - subquery_for_set.query_tree, + query_tree, subquery_options, planner_context->getGlobalPlannerContext()); subquery_planner.buildQueryPlanIfNeeded(); @@ -955,8 +955,6 @@ void addBuildSubqueriesForSetsStepIfNeeded( subquery->setQueryPlan(std::make_unique(std::move(subquery_planner).extractQueryPlan())); } - //addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext()); - if (!subqueries.empty()) { auto step = std::make_unique( diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 54d81dd7650..06099eef872 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -36,15 +36,13 @@ static ITransformingStep::Traits getTraits() CreatingSetStep::CreatingSetStep( const DataStream & input_stream_, - String description_, - SubqueryForSet & subquery_for_set_, - FutureSetPtr set_, + SetAndKeyPtr set_and_key_, + StoragePtr external_table_, SizeLimits network_transfer_limits_, ContextPtr context_) : ITransformingStep(input_stream_, Block{}, getTraits()) - , description(std::move(description_)) - , subquery_for_set(subquery_for_set_) - , set(std::move(set_)) + , set_and_key(std::move(set_and_key_)) + , external_table(std::move(external_table_)) , network_transfer_limits(std::move(network_transfer_limits_)) , context(std::move(context_)) { @@ -52,7 +50,7 @@ CreatingSetStep::CreatingSetStep( void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - pipeline.addCreatingSetsTransform(getOutputStream().header, subquery_for_set, std::move(set), network_transfer_limits, context->getPreparedSetsCache()); + pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(set_and_key), std::move(external_table), network_transfer_limits, context->getPreparedSetsCache()); } void CreatingSetStep::updateOutputStream() @@ -65,16 +63,16 @@ void CreatingSetStep::describeActions(FormatSettings & settings) const String prefix(settings.offset, ' '); settings.out << prefix; - if (subquery_for_set.set) + if (set_and_key->set) settings.out << "Set: "; - settings.out << description << '\n'; + settings.out << set_and_key->key << '\n'; } void CreatingSetStep::describeActions(JSONBuilder::JSONMap & map) const { - if (subquery_for_set.set) - map.add("Set", description); + if (set_and_key->set) + map.add("Set", set_and_key->key); } @@ -126,7 +124,7 @@ void CreatingSetsStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } -void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subqueries, ContextPtr context) +void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subquery, ContextPtr context) { DataStreams input_streams; input_streams.emplace_back(query_plan.getCurrentDataStream()); @@ -135,7 +133,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, std::vector(std::move(query_plan))); query_plan = QueryPlan(); - for (auto & future_set : sets_from_subqueries) + for (auto & future_set : sets_from_subquery) { if (future_set->get()) continue; @@ -193,7 +191,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, if (!prepared_sets) return; - auto subqueries = prepared_sets->detachSubqueries(); + auto subqueries = prepared_sets->getSubqueries(); if (subqueries.empty()) return; diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index c41bb782e0b..7110775da79 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -13,12 +13,11 @@ class CreatingSetStep : public ITransformingStep { public: CreatingSetStep( - const DataStream & input_stream_, - String description_, - SubqueryForSet & subquery_for_set_, - FutureSetPtr set_, - SizeLimits network_transfer_limits_, - ContextPtr context_); + const DataStream & input_stream_, + SetAndKeyPtr set_and_key_, + StoragePtr external_table_, + SizeLimits network_transfer_limits_, + ContextPtr context_); String getName() const override { return "CreatingSet"; } @@ -30,9 +29,8 @@ public: private: void updateOutputStream() override; - String description; - SubqueryForSet & subquery_for_set; - FutureSetPtr set; + SetAndKeyPtr set_and_key; + StoragePtr external_table; SizeLimits network_transfer_limits; ContextPtr context; }; diff --git a/src/Processors/Transforms/CreatingSetsTransform.cpp b/src/Processors/Transforms/CreatingSetsTransform.cpp index aa374446f3e..afce1355f7a 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -25,13 +25,13 @@ CreatingSetsTransform::~CreatingSetsTransform() = default; CreatingSetsTransform::CreatingSetsTransform( Block in_header_, Block out_header_, - SubqueryForSet & subquery_for_set_, - FutureSetPtr set_, + SetAndKeyPtr set_and_key_, + StoragePtr external_table_, SizeLimits network_transfer_limits_, PreparedSetsCachePtr prepared_sets_cache_) : IAccumulatingTransform(std::move(in_header_), std::move(out_header_)) - , subquery(subquery_for_set_) - , set(std::move(set_)) + , set_and_key(std::move(set_and_key_)) + , external_table(std::move(external_table_)) , network_transfer_limits(std::move(network_transfer_limits_)) , prepared_sets_cache(std::move(prepared_sets_cache_)) { @@ -54,31 +54,30 @@ void CreatingSetsTransform::work() void CreatingSetsTransform::startSubquery() { /// Lookup the set in the cache if we don't need to build table. - if (prepared_sets_cache && !subquery.table) + if (prepared_sets_cache && !external_table) { /// Try to find the set in the cache and wait for it to be built. /// Retry if the set from cache fails to be built. while (true) { - auto from_cache = prepared_sets_cache->findOrPromiseToBuild(subquery.key); + auto from_cache = prepared_sets_cache->findOrPromiseToBuild(set_and_key->key); if (from_cache.index() == 0) { - LOG_TRACE(log, "Building set, key: {}", subquery.key); + LOG_TRACE(log, "Building set, key: {}", set_and_key->key); promise_to_build = std::move(std::get<0>(from_cache)); } else { - LOG_TRACE(log, "Waiting for set to be build by another thread, key: {}", subquery.key); + LOG_TRACE(log, "Waiting for set to be build by another thread, key: {}", set_and_key->key); SharedSet set_built_by_another_thread = std::move(std::get<1>(from_cache)); const SetPtr & ready_set = set_built_by_another_thread.get(); if (!ready_set) { - LOG_TRACE(log, "Failed to use set from cache, key: {}", subquery.key); + LOG_TRACE(log, "Failed to use set from cache, key: {}", set_and_key->key); continue; } - //subquery.promise_to_fill_set.set_value(ready_set); - subquery.set = ready_set; //.reset(); + set_and_key->set = ready_set; done_with_set = true; set_from_cache = true; } @@ -86,21 +85,19 @@ void CreatingSetsTransform::startSubquery() } } - if (subquery.set && !set_from_cache) - LOG_TRACE(log, "Creating set, key: {}", subquery.key); - if (subquery.table) + if (set_and_key->set && !set_from_cache) + LOG_TRACE(log, "Creating set, key: {}", set_and_key->key); + if (external_table) LOG_TRACE(log, "Filling temporary table."); - // std::cerr << StackTrace().toString() << std::endl; - - if (subquery.table) + if (external_table) /// TODO: make via port - table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), nullptr, /*async_insert=*/false)); + table_out = QueryPipeline(external_table->write({}, external_table->getInMemoryMetadataPtr(), nullptr, /*async_insert=*/false)); - done_with_set = !subquery.set || set_from_cache; - done_with_table = !subquery.table; + done_with_set = !set_and_key->set || set_from_cache; + done_with_table = !external_table; - if ((done_with_set && !set_from_cache) /*&& done_with_join*/ && done_with_table) + if ((done_with_set && !set_from_cache) && done_with_table) throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: nothing to do with subquery"); if (table_out.initialized()) @@ -120,9 +117,9 @@ void CreatingSetsTransform::finishSubquery() } else if (read_rows != 0) { - if (subquery.set) - LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set->getTotalRowCount(), read_rows, seconds); - if (subquery.table) + if (set_and_key->set) + LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", set_and_key->set->getTotalRowCount(), read_rows, seconds); + if (external_table) LOG_DEBUG(log, "Created Table with {} rows in {} sec.", read_rows, seconds); } else @@ -135,12 +132,6 @@ void CreatingSetsTransform::init() { is_initialized = true; - // if (subquery.set) - // { - // //std::cerr << "=========== " << getInputPort().getHeader().dumpStructure() << std::endl; - // subquery.set->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName()); - // } - watch.restart(); startSubquery(); } @@ -152,7 +143,7 @@ void CreatingSetsTransform::consume(Chunk chunk) if (!done_with_set) { - if (!subquery.set->insertFromBlock(block.getColumnsWithTypeAndName())) + if (!set_and_key->set->insertFromBlock(block.getColumnsWithTypeAndName())) done_with_set = true; } @@ -175,12 +166,11 @@ void CreatingSetsTransform::consume(Chunk chunk) Chunk CreatingSetsTransform::generate() { - if (subquery.set && !set_from_cache) + if (set_and_key->set && !set_from_cache) { - subquery.set->finishInsert(); - //subquery.promise_to_fill_set.set_value(subquery.set); + set_and_key->set->finishInsert(); if (promise_to_build) - promise_to_build->set_value(subquery.set); + promise_to_build->set_value(set_and_key->set); } if (table_out.initialized()) diff --git a/src/Processors/Transforms/CreatingSetsTransform.h b/src/Processors/Transforms/CreatingSetsTransform.h index 27c330bdbc3..d1ec7dcbca7 100644 --- a/src/Processors/Transforms/CreatingSetsTransform.h +++ b/src/Processors/Transforms/CreatingSetsTransform.h @@ -29,8 +29,8 @@ public: CreatingSetsTransform( Block in_header_, Block out_header_, - SubqueryForSet & subquery_for_set_, - FutureSetPtr set_, + SetAndKeyPtr set_and_key_, + StoragePtr external_table_, SizeLimits network_transfer_limits_, PreparedSetsCachePtr prepared_sets_cache_); @@ -43,8 +43,8 @@ public: Chunk generate() override; private: - SubqueryForSet & subquery; - FutureSetPtr set; + SetAndKeyPtr set_and_key; + StoragePtr external_table; std::optional> promise_to_build; QueryPipeline table_out; diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index dca185cec8b..dedf85e409c 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -571,8 +571,8 @@ std::unique_ptr QueryPipelineBuilder::joinPipelinesRightLe void QueryPipelineBuilder::addCreatingSetsTransform( const Block & res_header, - SubqueryForSet & subquery_for_set, - FutureSetPtr set, + SetAndKeyPtr set_and_key, + StoragePtr external_table, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache) { @@ -581,8 +581,8 @@ void QueryPipelineBuilder::addCreatingSetsTransform( auto transform = std::make_shared( getHeader(), res_header, - subquery_for_set, - std::move(set), + std::move(set_and_key), + std::move(external_table), limits, std::move(prepared_sets_cache)); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 134de59520d..e744e3612ce 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -33,8 +33,8 @@ class TableJoin; class QueryPipelineBuilder; using QueryPipelineBuilderPtr = std::unique_ptr; -class FutureSet; -using FutureSetPtr = std::shared_ptr; +struct SetAndKey; +using SetAndKeyPtr = std::shared_ptr; class PreparedSetsCache; using PreparedSetsCachePtr = std::shared_ptr; @@ -146,8 +146,8 @@ public: void addCreatingSetsTransform( const Block & res_header, - SubqueryForSet & subquery_for_set, - FutureSetPtr set, + SetAndKeyPtr set_and_key, + StoragePtr external_table, const SizeLimits & limits, PreparedSetsCachePtr prepared_sets_cache); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c3cf146e88e..125fe98203f 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -978,7 +978,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (ranges.ranges.empty()) break; - auto & index_and_condition = skip_indexes.useful_indices[idx]; + const auto & index_and_condition = skip_indexes.useful_indices[idx]; auto & stat = useful_indices_stat[idx]; stat.total_parts.fetch_add(1, std::memory_order_relaxed); stat.total_granules.fetch_add(ranges.ranges.getNumberOfMarks(), std::memory_order_relaxed); @@ -1006,7 +1006,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (ranges.ranges.empty()) break; - auto & indices_and_condition = skip_indexes.merged_indices[idx]; + const auto & indices_and_condition = skip_indexes.merged_indices[idx]; auto & stat = merged_indices_stat[idx]; stat.total_parts.fetch_add(1, std::memory_order_relaxed); From 22e49748b5ff129422fca4ff2da92b26e38d3906 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Jun 2023 14:23:04 +0000 Subject: [PATCH 083/171] Cleanup. --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 4 +- src/Analyzer/SetUtils.cpp | 13 +- src/Analyzer/SetUtils.h | 10 +- src/Functions/in.cpp | 9 +- src/Interpreters/ActionsVisitor.cpp | 37 +- .../ClusterProxy/SelectStreamFactory.cpp | 8 +- .../ClusterProxy/SelectStreamFactory.h | 4 +- .../ClusterProxy/executeQuery.cpp | 1 - src/Interpreters/ExpressionAnalyzer.cpp | 119 ----- src/Interpreters/ExpressionAnalyzer.h | 8 - src/Interpreters/GlobalSubqueriesVisitor.h | 48 +- src/Interpreters/InterpreterSelectQuery.cpp | 3 - src/Interpreters/PreparedSets.cpp | 493 ++++++------------ src/Interpreters/PreparedSets.h | 65 ++- src/Interpreters/Set.cpp | 13 +- src/Interpreters/Set.h | 22 +- src/Interpreters/SetKeys.h | 18 + src/Planner/CollectSets.cpp | 18 +- src/Planner/CollectSets.h | 2 +- src/Planner/Planner.cpp | 9 +- src/Planner/PlannerActionsVisitor.cpp | 6 +- src/Planner/PlannerContext.cpp | 46 -- src/Planner/PlannerContext.h | 72 +-- src/Planner/PlannerJoinTree.cpp | 21 +- src/Planner/Utils.cpp | 3 +- src/Planner/Utils.h | 1 - .../CreateSetAndFilterOnTheFlyStep.cpp | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 18 +- src/Processors/QueryPlan/CreatingSetsStep.h | 10 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 3 +- .../QueryPlan/DistributedCreateLocalPlan.h | 1 - .../QueryPlan/ReadFromMergeTree.cpp | 27 - src/Processors/QueryPlan/ReadFromRemote.cpp | 6 +- src/Processors/QueryPlan/ReadFromRemote.h | 2 - src/Storages/KVStorageUtils.cpp | 1 - src/Storages/MergeTree/KeyCondition.cpp | 33 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- .../MergeTreeIndexConditionBloomFilter.cpp | 44 +- .../MergeTree/MergeTreeIndexFullText.cpp | 9 +- .../MergeTree/MergeTreeIndexInverted.cpp | 14 +- src/Storages/MergeTree/PartitionPruner.cpp | 9 - src/Storages/StorageDistributed.cpp | 5 +- src/Storages/StorageMergeTree.cpp | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 3 +- src/Storages/StorageSet.cpp | 4 +- .../System/StorageSystemZooKeeper.cpp | 10 +- src/Storages/VirtualColumnUtils.cpp | 20 - src/Storages/buildQueryTreeForShard.cpp | 17 +- 48 files changed, 343 insertions(+), 955 deletions(-) create mode 100644 src/Interpreters/SetKeys.h diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 98f40dc4c15..163092f1b7f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5211,14 +5211,14 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi const auto & settings = scope.context->getSettingsRef(); - auto result_block = makeSetForConstantValue(first_argument_constant_type, + auto result_block = getSetElementsForConstantValue(first_argument_constant_type, second_argument_constant_literal, second_argument_constant_type, settings.transform_null_in); 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, false /*fill_set_elements*/, 0, settings.transform_null_in); + 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()); diff --git a/src/Analyzer/SetUtils.cpp b/src/Analyzer/SetUtils.cpp index 2fb05449714..15eec16e899 100644 --- a/src/Analyzer/SetUtils.cpp +++ b/src/Analyzer/SetUtils.cpp @@ -118,7 +118,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes & } -Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in) +Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in) { DataTypes set_element_types = {expression_type}; const auto * lhs_tuple_type = typeid_cast(expression_type.get()); @@ -135,9 +135,6 @@ Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & size_t lhs_type_depth = getCompoundTypeDepth(*expression_type); size_t rhs_type_depth = getCompoundTypeDepth(*value_type); - // SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode}; - // bool transform_null_in = settings.transform_null_in; - Block result_block; if (lhs_type_depth == rhs_type_depth) @@ -171,14 +168,6 @@ Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & } return result_block; - - // auto set = std::make_shared(size_limits_for_set, true /*fill_set_elements*/, tranform_null_in); - - // set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName()); - // set->insertFromBlock(result_block.getColumnsWithTypeAndName()); - // set->finishInsert(); - - // return set; } } diff --git a/src/Analyzer/SetUtils.h b/src/Analyzer/SetUtils.h index fdeaa4a3c48..c35b45dce59 100644 --- a/src/Analyzer/SetUtils.h +++ b/src/Analyzer/SetUtils.h @@ -12,18 +12,12 @@ namespace DB class Set; using SetPtr = std::shared_ptr; -/** Make set for constant part of IN subquery. +/** Get set elements for constant part of IN subquery. * Throws exception if parameters are not valid for IN function. * * Example: SELECT id FROM test_table WHERE id IN (1, 2, 3, 4); * Example: SELECT id FROM test_table WHERE id IN ((1, 2), (3, 4)); - * - * @param expression_type - type of first argument of function IN. - * @param value - constant value of second argument of function IN. - * @param value_type - type of second argument of function IN. - * - * @return SetPtr for constant value. */ -Block makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in); +Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in); } diff --git a/src/Functions/in.cpp b/src/Functions/in.cpp index c19d1e72003..9045ba677f2 100644 --- a/src/Functions/in.cpp +++ b/src/Functions/in.cpp @@ -123,9 +123,12 @@ public: } auto future_set = column_set->getData(); - auto set = future_set ? future_set->get() : nullptr; - if (!future_set || !set) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName()); + if (!future_set) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No Set is passed as the second argument for function '{}'", getName()); + + auto set = future_set->get(); + if (!set) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set is passed as the second argument for function '{}'", getName()); auto set_types = set->getDataTypes(); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 7c3af4ce691..bf78868463a 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -445,19 +445,16 @@ FutureSetPtr makeExplicitSet( if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - for (auto & element_type : set_element_types) - if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - element_type = low_cardinality_type->getDictionaryType(); - - // if (!set_element_types.empty()) - // std::cerr << "========== " << set_element_types[0]->getName() << std::endl; - auto set_element_keys = Set::getElementTypes(set_element_types, context->getSettingsRef().transform_null_in); auto set_key = right_arg->getTreeHash(); if (auto set = prepared_sets.findTuple(set_key, set_element_keys)) return set; /// Already prepared. + for (auto & element_type : set_element_types) + if (const auto * low_cardinality_type = typeid_cast(element_type.get())) + element_type = low_cardinality_type->getDictionaryType(); + Block block; const auto & right_arg_func = std::dynamic_pointer_cast(right_arg); if (right_arg_func && (right_arg_func->name == "tuple" || right_arg_func->name == "array")) @@ -1400,11 +1397,21 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool { if (no_subqueries) return {}; - //std::cerr << queryToString(right_in_operand) << std::endl; PreparedSets::Hash set_key; if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier) { + /// Here we can be only from mutation interpreter. Normal selects with analyzed use other interpreter. + /// This is a hacky way to allow reusing cache for prepared sets. + /// + /// Mutation is executed in two stages: + /// * first, query 'SELECT count() FROM table WHERE ...' is executed to get the set of affected parts (using analyzer) + /// * second, every part is mutated separately, where plan is build "manually", usign this code as well + /// To share the Set in between first and second stage, we should use the same hash. + /// New analyzer is uses a hash from query tree, so here we also build a query tree. + /// + /// Note : this code can be safely removed, but the test 02581_share_big_sets will be too slow (and fail by timeout). + /// Note : we should use new analyzer for mutations and remove this hack. InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery()); const auto & query_tree = interpreter.getQueryTree(); if (auto * query_node = query_tree->as()) @@ -1414,12 +1421,6 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool else set_key = right_in_operand->getTreeHash(); - // std::cerr << set_key.toString() << std::endl; - // std::cerr << data.prepared_sets->getSets().size() << std::endl; - // std::cerr << reinterpret_cast(data.prepared_sets.get()) << std::endl; - // for (const auto & [k, v] : data.prepared_sets->getSets()) - // std::cerr << "... " << k.toString(); - if (auto set = data.prepared_sets->findSubquery(set_key)) return set; @@ -1440,12 +1441,12 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool if (!data.getContext()->isGlobalContext()) { - // std::cerr << ".... checking for " << identifier->getColumnName() << std::endl; + /// If we are reading from storage, it can be an external table which is used for GLOBAL IN. + /// Here, we take FutureSet which is used to build external table. + /// It will be used if set is useful for primary key. During PK analysis + /// temporary table is not filled yet, so we need to fill it first. if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName())) - { external_table_set = tmp_table->future_set; - // std::cerr << "Found " << reinterpret_cast(tmp_table.get()) << " " << reinterpret_cast(external_table_set.get()) << std::endl; - } } } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 0f8a725e144..0cf3f360994 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -92,13 +92,11 @@ SelectStreamFactory::SelectStreamFactory( const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, const StorageSnapshotPtr & storage_snapshot_, - QueryProcessingStage::Enum processed_stage_, - PreparedSetsPtr prepared_sets_) + QueryProcessingStage::Enum processed_stage_) : header(header_), objects_by_shard(objects_by_shard_), storage_snapshot(storage_snapshot_), - processed_stage(processed_stage_), - prepared_sets(std::move(prepared_sets_)) + processed_stage(processed_stage_) { } @@ -119,7 +117,7 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { local_plans.emplace_back(createLocalPlan( - query_ast, header, context, processed_stage, prepared_sets, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); + query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); }; auto emplace_remote_stream = [&](bool lazy = false, time_t local_delay = 0) diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index b19012ddba6..030c0b77dd5 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -69,8 +69,7 @@ public: const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, const StorageSnapshotPtr & storage_snapshot_, - QueryProcessingStage::Enum processed_stage_, - PreparedSetsPtr prepared_sets_); + QueryProcessingStage::Enum processed_stage_); void createForShard( const Cluster::ShardInfo & shard_info, @@ -95,7 +94,6 @@ public: const ColumnsDescriptionByShardNum objects_by_shard; const StorageSnapshotPtr storage_snapshot; QueryProcessingStage::Enum processed_stage; - PreparedSetsPtr prepared_sets; }; } diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index a9cf3d55392..e2f1dfe8ba7 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -226,7 +226,6 @@ void executeQuery( std::move(remote_shards), header, processed_stage, - stream_factory.prepared_sets, main_table, table_func_ptr, new_context, diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index b05563cf29f..f27d23e8e94 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -450,77 +450,6 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global, b } -// void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options) -// { -// if (!prepared_sets) -// return; - -// auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); - -// if (prepared_sets->getFuture(set_key).isValid()) -// return; /// Already prepared. - -// if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name)) -// { -// prepared_sets->set(set_key, set_ptr_from_storage_set); -// return; -// } - -// auto build_set = [&] () -> SetPtr -// { -// LOG_TRACE(getLogger(), "Building set, key: {}", set_key.toString()); - -// auto interpreter_subquery = interpretSubquery(subquery_or_table_name, getContext(), {}, query_options); -// auto io = interpreter_subquery->execute(); -// PullingAsyncPipelineExecutor executor(io.pipeline); - -// SetPtr set = std::make_shared(settings.size_limits_for_set_used_with_index, true, getContext()->getSettingsRef().transform_null_in); -// set->setHeader(executor.getHeader().getColumnsWithTypeAndName()); - -// Block block; -// while (executor.pull(block)) -// { -// if (block.rows() == 0) -// continue; - -// /// If the limits have been exceeded, give up and let the default subquery processing actions take place. -// if (!set->insertFromBlock(block.getColumnsWithTypeAndName())) -// return nullptr; -// } - -// set->finishInsert(); - -// return set; -// }; - -// SetPtr set; - -// auto set_cache = getContext()->getPreparedSetsCache(); -// if (set_cache) -// { -// auto from_cache = set_cache->findOrPromiseToBuild(set_key.toString()); -// if (from_cache.index() == 0) -// { -// set = build_set(); -// std::get<0>(from_cache).set_value(set); -// } -// else -// { -// LOG_TRACE(getLogger(), "Waiting for set, key: {}", set_key.toString()); -// set = std::get<1>(from_cache).get(); -// } -// } -// else -// { -// set = build_set(); -// } - -// if (!set) -// return; - -// prepared_sets->set(set_key, std::move(set)); -// } - SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name) { const auto * table = subquery_or_table_name->as(); @@ -534,54 +463,6 @@ SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_o return storage_set->getSet(); } - -/// Performance optimization for IN() if storage supports it. -// void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) -// { -// if (!node || !storage() || !storage()->supportsIndexForIn()) -// return; - -// for (auto & child : node->children) -// { -// /// Don't descend into subqueries. -// if (child->as()) -// continue; - -// /// Don't descend into lambda functions -// const auto * func = child->as(); -// if (func && func->name == "lambda") -// continue; - -// makeSetsForIndex(child); -// } - -// const auto * func = node->as(); -// if (func && functionIsInOrGlobalInOperator(func->name)) -// { -// const IAST & args = *func->arguments; -// const ASTPtr & left_in_operand = args.children.at(0); - -// if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot)) -// { -// const ASTPtr & arg = args.children.at(1); -// if (arg->as() || arg->as()) -// { -// if (settings.use_index_for_in_with_subqueries) -// tryMakeSetForIndexFromSubquery(arg, query_options); -// } -// else -// { -// auto temp_actions = std::make_shared(columns_after_join); -// getRootActions(left_in_operand, true, temp_actions); - -// if (prepared_sets && temp_actions->tryFindInOutputs(left_in_operand->getColumnName())) -// makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, *prepared_sets); -// } -// } -// } -// } - - void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts) { LogAST log; diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index e4a9e96c517..271c3943afc 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -141,11 +141,6 @@ public: void makeWindowDescriptionFromAST(const Context & context, const WindowDescriptions & existing_descriptions, WindowDescription & desc, const IAST * ast); void makeWindowDescriptions(ActionsDAGPtr actions); - /** Create Set from a subquery or a table expression in the query. The created set is suitable for using the index. - * The set will not be created if its size hits the limit. - */ - // void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {}); - /** Checks if subquery is not a plain StorageSet. * Because while making set we will read data from StorageSet which is not allowed. * Returns valid SetPtr from StorageSet if the latter is used after IN or nullptr otherwise. @@ -363,9 +358,6 @@ public: /// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases. ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const; - /// Create Set-s that we make from IN section to use index on them. - // void makeSetsForIndex(const ASTPtr & node); - private: StorageMetadataPtr metadata_snapshot; /// If non-empty, ignore all expressions not from this list. diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index 8c784d3c2ff..5b633fee9b6 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -69,7 +69,6 @@ public: void addExternalStorage(ASTPtr & ast, const Names & required_columns, bool set_alias = false) { - // std::cerr << "=============== addExternalStorage is remote " << is_remote << std::endl; /// With nondistributed queries, creating temporary tables does not make sense. if (!is_remote) return; @@ -163,49 +162,20 @@ public: nullptr, /*create_for_global_subquery*/ true); StoragePtr external_storage = external_storage_holder->getTable(); - - // std::cerr << "......... adding external table " << external_table_name << std::endl; - external_tables.emplace(external_table_name, external_storage_holder); - /// We need to materialize external tables immediately because reading from distributed - /// tables might generate local plans which can refer to external tables during index - /// analysis. It's too late to populate the external table via CreatingSetsTransform. - // if (is_explain) - // { - // /// Do not materialize external tables if it's explain statement. - // } - // else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries) - // { - // auto external_table = external_storage_holder->getTable(); - // auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext()); - // auto io = interpreter->execute(); - // io.pipeline.complete(std::move(table_out)); - // CompletedPipelineExecutor executor(io.pipeline); - // executor.execute(); - // } - // else + auto set_key = database_and_table_name->getTreeHash(); + + if (!prepared_sets->findSubquery(set_key)) { - // auto & subquery_for_set = prepared_sets->getSubquery(external_table_name); - // subquery_for_set.createSource(*interpreter, external_storage); - auto set_key = database_and_table_name->getTreeHash(); + std::unique_ptr source = std::make_unique(); + interpreter->buildQueryPlan(*source); - // std::cerr << "====== Adding key " << set_key.toString() << std::endl; - - if (!prepared_sets->findSubquery(set_key)) - { - std::unique_ptr source = std::make_unique(); - interpreter->buildQueryPlan(*source); - - //std::cerr << reinterpret_cast(prepared_sets.get()) << std::endl; - auto future_set = prepared_sets->addFromSubquery(set_key, std::move(source), std::move(external_storage), nullptr, getContext()->getSettingsRef()); - // std::cerr << "... Future set " << reinterpret_cast(external_storage_holder.get()) << " " << reinterpret_cast(future_set.get()) << std::endl; - external_storage_holder->future_set = std::move(future_set); - } - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Set is already created for GLOBAL IN"); - //prepared_sets->addStorageToSubquery(key, std::move(external_storage)); + auto future_set = prepared_sets->addFromSubquery(set_key, std::move(source), std::move(external_storage), nullptr, getContext()->getSettingsRef()); + external_storage_holder->future_set = std::move(future_set); } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Set is already created for GLOBAL IN"); /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, * then a new temporary table will be created (for example, _data1), diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 85c3457c6b9..6ea15312ec4 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -953,10 +953,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl() if (storage && !options.only_analyze) { - // query_analyzer->makeSetsForIndex(select_query.where()); - // query_analyzer->makeSetsForIndex(select_query.prewhere()); query_info.prepared_sets = query_analyzer->getPreparedSets(); - from_stage = storage->getQueryProcessingStage(context, options.to_stage, storage_snapshot, query_info); } diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 2aa3e02c713..428ef873bc5 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -25,20 +25,25 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -class FutureSetFromTuple final : public FutureSet +static SizeLimits getSizeLimitsForSet(const Settings & settings) { -public: - FutureSetFromTuple(Block block, const Settings & settings); + return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); +} - SetPtr get() const override { return set; } - SetPtr buildOrderedSetInplace(const ContextPtr & context) override; +static bool equals(const DataTypes & lhs, const DataTypes & rhs) +{ + size_t size = lhs.size(); + if (size != rhs.size()) + return false; - const DataTypes & getTypes() const override { return set->getElementsTypes(); } + for (size_t i = 0; i < size; ++i) + { + if (!lhs[i]->equals(*rhs[i])) + return false; + } -private: - SetPtr set; - SetKeyColumns set_key_columns; -}; + return true; +} FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {} @@ -51,42 +56,153 @@ SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &) } -// PreparedSetKey PreparedSetKey::forLiteral(Hash hash, DataTypes types_) -// { -// /// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now, -// /// just converts LowCardinality to ordinary types. -// for (auto & type : types_) -// type = recursiveRemoveLowCardinality(type); +FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) +{ + 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->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); -// PreparedSetKey key; -// key.ast_hash = hash; -// key.types = std::move(types_); -// return key; -// } + Columns columns; + columns.reserve(block.columns()); + for (const auto & column : block) + columns.emplace_back(column.column); -// PreparedSetKey PreparedSetKey::forSubquery(Hash hash) -// { -// PreparedSetKey key; -// key.ast_hash = hash; -// return key; -// } + set_key_columns.filter = ColumnUInt8::create(block.rows()); -// bool PreparedSetKey::operator==(const PreparedSetKey & other) const -// { -// if (ast_hash != other.ast_hash) -// return false; + set->insertFromColumns(columns, set_key_columns); + set->finishInsert(); +} -// if (types.size() != other.types.size()) -// return false; +const DataTypes & FutureSetFromTuple::getTypes() const { return set->getElementsTypes(); } -// for (size_t i = 0; i < types.size(); ++i) -// { -// if (!types[i]->equals(*other.types[i])) -// return false; -// } +SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) +{ + if (set->hasExplicitSetElements()) + return set; + + const auto & settings = context->getSettingsRef(); + size_t max_values = settings.use_index_for_in_with_subqueries_max_values; + bool too_many_values = max_values && max_values < set->getTotalRowCount(); + if (!too_many_values) + { + set->fillSetElements(); + set->appendSetElements(set_key_columns); + } + + return set; +} + + +FutureSetFromSubquery::FutureSetFromSubquery( + String key, + std::unique_ptr source_, + StoragePtr external_table_, + FutureSetPtr external_table_set_, + const Settings & settings) + : 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); + + 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->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); +} + +FutureSetFromSubquery::FutureSetFromSubquery( + String key, + QueryTreeNodePtr query_tree_, + const Settings & settings) + : 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); +} + +SetPtr FutureSetFromSubquery::get() const +{ + if (set_and_key->set != nullptr && set_and_key->set->isCreated()) + return set_and_key->set; + + return nullptr; +} + +void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source_) +{ + source = std::move(source_); + set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); +} + +const DataTypes & FutureSetFromSubquery::getTypes() const +{ + return set_and_key->set->getElementsTypes(); +} + +std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) +{ + if (set_and_key->set->isCreated()) + return nullptr; + + const auto & settings = context->getSettingsRef(); + + auto plan = std::move(source); + + if (!plan) + return nullptr; + + auto creating_set = std::make_unique( + plan->getCurrentDataStream(), + set_and_key, + external_table, + SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), + context); + creating_set->setStepDescription("Create set for subquery"); + plan->addStep(std::move(creating_set)); + return plan; +} + +SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) +{ + if (!context->getSettingsRef().use_index_for_in_with_subqueries) + return nullptr; + + if (auto set = get()) + { + if (set->hasExplicitSetElements()) + return set; + + return nullptr; + } + + if (external_table_set) + { + auto set = external_table_set->buildOrderedSetInplace(context); + if (set) + return set_and_key->set = set; + } + + auto plan = build(context); + if (!plan) + return nullptr; + + set_and_key->set->fillSetElements(); + auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + pipeline.complete(std::make_shared(Block())); + + CompletedPipelineExecutor executor(pipeline); + executor.execute(); + + set_and_key->set->checkIsCreated(); + + return set_and_key->set; +} -// return true; -// } String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & types) { @@ -108,84 +224,18 @@ String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & return buf.str(); } -// SubqueryForSet & PreparedSets::createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key, -// SizeLimits set_size_limit, bool transform_null_in) -// { -// SubqueryForSet & subquery = subqueries[subquery_id]; - -// /// If you already created a Set with the same subquery / table for another ast -// /// In that case several PreparedSetKey would share same subquery and set -// /// Not sure if it's really possible case (maybe for distributed query when set was filled by external table?) -// if (subquery.set.isValid()) -// sets[key] = subquery.set; -// else -// { -// subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); -// sets[key] = FutureSet(subquery.promise_to_fill_set.get_future()); -// } - -// if (!subquery.set_in_progress) -// { -// subquery.key = key.toString(); -// subquery.set_in_progress = std::make_shared(set_size_limit, false, transform_null_in); -// } - -// return subquery; -// } - -/// If the subquery is not associated with any set, create default-constructed SubqueryForSet. -/// It's aimed to fill external table passed to SubqueryForSet::createSource. -// void PreparedSets::addStorageToSubquery(const String & subquery_id, StoragePtr storage) -// { -// auto it = subqueries.find(subquery_id); -// if (it == subqueries.end()) -// throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find subquery {}", subquery_id); - -// it->second->addStorage(std::move(storage)); -// } -static bool equals(const DataTypes & lhs, const DataTypes & rhs) -{ - size_t size = lhs.size(); - if (size != rhs.size()) - return false; - - for (size_t i = 0; i < size; ++i) - { - if (!lhs[i]->equals(*rhs[i])) - return false; - } - - return true; -} - -static bool tryInsertSet(std::vector> & sets, FutureSetPtr new_set) -{ - auto types = new_set->getTypes(); - for (const auto & set : sets) - if (equals(set->getTypes(), new_set->getTypes())) - return false; - - sets.push_back(std::move(new_set)); - return true; -} - -static FutureSetPtr findSet(const std::vector> & sets, const DataTypes & types) -{ - for (const auto & set : sets) - if (equals(set->getTypes(), types)) - return set; - - return nullptr; -} - FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings) { auto from_tuple = std::make_shared(std::move(block), settings); + const auto & set_types = from_tuple->getTypes(); auto & sets_by_hash = sets_from_tuple[key]; - if (!tryInsertSet(sets_by_hash, from_tuple)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_tuple->getTypes())); + auto types = from_tuple->getTypes(); + for (const auto & set : sets_by_hash) + if (equals(set->getTypes(), set_types)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_tuple->getTypes())); + sets_by_hash.push_back(std::move(from_tuple)); return from_tuple; } @@ -246,7 +296,11 @@ FutureSetPtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) if (it == sets_from_tuple.end()) return nullptr; - return findSet(it->second, types); + for (const auto & set : it->second) + if (equals(set->getTypes(), types)) + return set; + + return nullptr; } std::shared_ptr PreparedSets::findSubquery(const Hash & key) const @@ -267,36 +321,9 @@ std::shared_ptr PreparedSets::findStorage(const Hash & key return it->second; } -// FutureSetPtr PreparedSets::getFuture(const PreparedSetKey & key) const -// { -// auto it = sets.find(key); -// if (it == sets.end()) -// return {}; -// return it->second; -// } - -// SetPtr PreparedSets::get(const PreparedSetKey & key) const -// { -// auto it = sets.find(key); -// if (it == sets.end() || !it->second.isReady()) -// return nullptr; -// return it->second.get(); -// } - -// std::vector PreparedSets::getByTreeHash(IAST::Hash ast_hash) const -// { -// std::vector res; -// for (const auto & it : this->sets) -// { -// if (it.first.ast_hash == ast_hash) -// res.push_back(it.second); -// } -// return res; -// } - -std::vector> PreparedSets::getSubqueries() +PreparedSets::Subqueries PreparedSets::getSubqueries() { - std::vector> res; + PreparedSets::Subqueries res; res.reserve(sets_from_subqueries.size()); for (auto & [_, set] : sets_from_subqueries) res.push_back(set); @@ -304,33 +331,10 @@ std::vector> PreparedSets::getSubqueries( return res; } -// void SubqueryForSet::createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_) -// { -// source = std::make_unique(); -// interpreter.buildQueryPlan(*source); -// if (table_) -// table = table_; -// } - -// bool SubqueryForSet::hasSource() const -// { -// return source != nullptr; -// } - -// QueryPlanPtr SubqueryForSet::detachSource() -// { -// auto res = std::move(source); -// source = nullptr; -// return res; -// } - - std::variant, SharedSet> PreparedSetsCache::findOrPromiseToBuild(const String & key) { std::lock_guard lock(cache_mutex); - // std::cerr << "PreparedSetsCache::findOrPromiseToBuild " << key << "\n" << StackTrace().toString() << std::endl; - auto it = cache.find(key); if (it != cache.end()) { @@ -347,163 +351,4 @@ std::variant, SharedSet> PreparedSetsCache::findOrPromiseTo return promise_to_fill_set; } -SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context) -{ - if (!context->getSettingsRef().use_index_for_in_with_subqueries) - return nullptr; - - if (auto set = get()) - { - if (set->hasExplicitSetElements()) - return set; - - return nullptr; - } - - if (external_table_set) - { - auto set = external_table_set->buildOrderedSetInplace(context); - if (set) - return set_and_key->set = set; - } - - auto plan = buildPlan(context); - if (!plan) - return nullptr; - - set_and_key->set->fillSetElements(); - set_and_key->set->initSetElements(); - auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)); - auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); - pipeline.complete(std::make_shared(Block())); - - CompletedPipelineExecutor executor(pipeline); - executor.execute(); - - set_and_key->set->checkIsCreated(); - - return set_and_key->set; -} - -SetPtr FutureSetFromSubquery::get() const -{ - if (set_and_key->set != nullptr && set_and_key->set->isCreated()) - return set_and_key->set; - - return nullptr; -} - -std::unique_ptr FutureSetFromSubquery::build(const ContextPtr & context) -{ - return buildPlan(context); -} - -static SizeLimits getSizeLimitsForSet(const Settings & settings) -{ - return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode); -} - -std::unique_ptr FutureSetFromSubquery::buildPlan(const ContextPtr & context) -{ - if (set_and_key->set->isCreated()) - return nullptr; - - const auto & settings = context->getSettingsRef(); - - auto plan = std::move(source); - - if (!plan) - return nullptr; - - auto creating_set = std::make_unique( - plan->getCurrentDataStream(), - set_and_key, - external_table, - SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), - context); - creating_set->setStepDescription("Create set for subquery"); - plan->addStep(std::move(creating_set)); - return plan; -} - -FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings) -{ - bool create_ordered_set = false; - auto size_limits = getSizeLimitsForSet(settings); - set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); - set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName()); - - Columns columns; - columns.reserve(block.columns()); - for (const auto & column : block) - columns.emplace_back(column.column); - - set_key_columns.filter = ColumnUInt8::create(block.rows()); - - set->insertFromColumns(columns, set_key_columns); - set->finishInsert(); -} - -FutureSetFromSubquery::FutureSetFromSubquery( - String key, - std::unique_ptr source_, - StoragePtr external_table_, - FutureSetPtr external_table_set_, - const Settings & settings) - : 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); - - bool create_ordered_set = false; - auto size_limits = getSizeLimitsForSet(settings); - set_and_key->set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); - set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); -} - -FutureSetFromSubquery::FutureSetFromSubquery( - String key, - QueryTreeNodePtr query_tree_, - //FutureSetPtr external_table_set_, - const Settings & settings) - : query_tree(std::move(query_tree_)) -{ - set_and_key = std::make_shared(); - set_and_key->key = std::move(key); - - bool create_ordered_set = false; - auto size_limits = getSizeLimitsForSet(settings); - set_and_key->set = std::make_shared(size_limits, create_ordered_set, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in); -} - -void FutureSetFromSubquery::setQueryPlan(std::unique_ptr source_) -{ - source = std::move(source_); - set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName()); -} - -const DataTypes & FutureSetFromSubquery::getTypes() const -{ - return set_and_key->set->getElementsTypes(); -} - -SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context) -{ - if (set->hasExplicitSetElements()) - return set; - - const auto & settings = context->getSettingsRef(); - size_t max_values = settings.use_index_for_in_with_subqueries_max_values; - bool too_many_values = max_values && max_values < set->getTotalRowCount(); - if (!too_many_values) - { - set->initSetElements(); - set->appendSetElements(set_key_columns); - } - - return set; -} - }; diff --git a/src/Interpreters/PreparedSets.h b/src/Interpreters/PreparedSets.h index 339f4a6b435..cb240f5260a 100644 --- a/src/Interpreters/PreparedSets.h +++ b/src/Interpreters/PreparedSets.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include namespace DB { @@ -17,7 +17,7 @@ class QueryPlan; class Set; using SetPtr = std::shared_ptr; -class InterpreterSelectWithUnionQuery; +struct SetKeyColumns; class IQueryTreeNode; using QueryTreeNodePtr = std::shared_ptr; @@ -44,14 +44,19 @@ class FutureSet public: virtual ~FutureSet() = default; + /// Returns set if set is ready (created and filled) or nullptr if not. virtual SetPtr get() const = 0; + /// Returns set->getElementsTypes(), even if set is not created yet. virtual const DataTypes & getTypes() const = 0; + /// If possible, return set with stored elements useful for PK analysis. virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0; }; using FutureSetPtr = std::shared_ptr; -class FutureSetFromStorage : public FutureSet +/// Future set from already filled set. +/// Usually it is from StorageSet. +class FutureSetFromStorage final : public FutureSet { public: FutureSetFromStorage(SetPtr set_); @@ -64,34 +69,27 @@ private: SetPtr set; }; +/// Set from tuple is filled as well as set from storage. +/// Additionally, it can be converted to set useful for PK. +class FutureSetFromTuple final : public FutureSet +{ +public: + FutureSetFromTuple(Block block, const Settings & settings); -/// Information on how to build set for the [GLOBAL] IN section. -// class SubqueryForSet -// { -// public: + SetPtr get() const override { return set; } + SetPtr buildOrderedSetInplace(const ContextPtr & context) override; -// void createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_ = nullptr); + const DataTypes & getTypes() const override; -// bool hasSource() const; +private: + SetPtr set; + SetKeyColumns set_key_columns; +}; -// /// Returns query plan for the set's source -// /// and removes it from SubqueryForSet because we need to build it only once. -// std::unique_ptr detachSource(); - -// /// Build this set from the result of the subquery. -// String key; -// SetPtr set; - -// /// If set, put the result into the table. -// /// This is a temporary table for transferring to remote servers for distributed query processing. -// StoragePtr table; - -// /// The source is obtained using the InterpreterSelectQuery subquery. -// std::unique_ptr source; -// QueryTreeNodePtr query_tree; -// }; - -class FutureSetFromSubquery : public FutureSet +/// Set from subquery can be built inplace for PK or in CreatingSet step. +/// If use_index_for_in_with_subqueries_max_values is reached, set for PK won't be created, +/// but ordinary set would be created instead. +class FutureSetFromSubquery final : public FutureSet { public: FutureSetFromSubquery( @@ -104,7 +102,6 @@ public: FutureSetFromSubquery( String key, QueryTreeNodePtr query_tree_, - //FutureSetPtr external_table_set_, const Settings & settings); SetPtr get() const override; @@ -123,10 +120,9 @@ private: std::unique_ptr source; QueryTreeNodePtr query_tree; - - std::unique_ptr buildPlan(const ContextPtr & context); }; +/// Container for all the sets used in query. class PreparedSets { public: @@ -137,7 +133,7 @@ public: UInt64 operator()(const Hash & key) const { return key.first ^ key.second; } }; - using SetsFromTuple = std::unordered_map>, Hashing>; + using SetsFromTuple = std::unordered_map>, Hashing>; using SetsFromStorage = std::unordered_map, Hashing>; using SetsFromSubqueries = std::unordered_map, Hashing>; @@ -160,11 +156,12 @@ public: std::shared_ptr findStorage(const Hash & key) const; std::shared_ptr findSubquery(const Hash & key) const; - std::vector> getSubqueries(); + using Subqueries = std::vector>; + Subqueries getSubqueries(); const SetsFromTuple & getSetsFromTuple() const { return sets_from_tuple; } - const SetsFromStorage & getSetsFromStorage() const { return sets_from_storage; } - const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; } + // const SetsFromStorage & getSetsFromStorage() const { return sets_from_storage; } + // const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; } static String toString(const Hash & key, const DataTypes & types); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 17df5064642..b42ff102f72 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -167,19 +167,16 @@ void Set::setHeader(const ColumnsWithTypeAndName & header) extractNestedColumnsAndNullMap(key_columns, null_map); } - if (fill_set_elements) - { - /// Create empty columns with set values in advance. - /// It is needed because set may be empty, so method 'insertFromBlock' will be never called. - initSetElements(); - } - /// Choose data structure to use for the set. data.init(data.chooseMethod(key_columns, key_sizes)); } -void Set::initSetElements() +void Set::fillSetElements() { + if (data.getTotalRowCount()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert set to ordered because it is not empty"); + + fill_set_elements = true; set_elements.reserve(keys_size); for (const auto & type : set_elements_types) set_elements.emplace_back(type->createColumn()); diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index d1218830969..9ea46e117ef 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -21,17 +22,6 @@ using FunctionBasePtr = std::shared_ptr; class Chunk; -/// Prepared key columns for set which can be added to fill set elements. -/// Used only to upgrade set from tuple. -struct SetKeyColumns -{ - /// The constant columns to the right of IN are not supported directly. For this, they first materialize. - ColumnRawPtrs key_columns; - Columns materialized_columns; - ColumnPtr null_map_holder; - ColumnUInt8::MutablePtr filter; -}; - /** Data structure for implementation of IN expression. */ class Set @@ -41,14 +31,12 @@ public: /// (that is useful only for checking that some value is in the set and may not store the original values), /// store all set elements in explicit form. /// This is needed for subsequent use for index. - Set(const SizeLimits & limits_, bool fill_set_elements_, size_t max_elements_to_fill_, bool transform_null_in_) + Set(const SizeLimits & limits_, size_t max_elements_to_fill_, bool transform_null_in_) : log(&Poco::Logger::get("Set")), - limits(limits_), fill_set_elements(fill_set_elements_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_) + limits(limits_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_) { } - void fillSetElements() { fill_set_elements = true; } - /** Set can be created either from AST or from a stream of data (subquery result). */ @@ -61,7 +49,7 @@ public: bool insertFromColumns(const Columns & columns); bool insertFromBlock(const ColumnsWithTypeAndName & columns); - void initSetElements(); + void fillSetElements(); bool insertFromColumns(const Columns & columns, SetKeyColumns & holder); void appendSetElements(SetKeyColumns & holder); @@ -129,7 +117,7 @@ private: 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; + bool fill_set_elements = false; size_t max_elements_to_fill; /// If true, insert NULL values to set. diff --git a/src/Interpreters/SetKeys.h b/src/Interpreters/SetKeys.h new file mode 100644 index 00000000000..2cab9849c9b --- /dev/null +++ b/src/Interpreters/SetKeys.h @@ -0,0 +1,18 @@ +#pragma once +#include + +namespace DB +{ + +/// Prepared key columns for set which can be added to fill set elements. +/// Used only to upgrade set from tuple. +struct SetKeyColumns +{ + /// The constant columns to the right of IN are not supported directly. For this, they first materialize. + ColumnRawPtrs key_columns; + Columns materialized_columns; + ColumnPtr null_map_holder; + ColumnUInt8::MutablePtr filter; +}; + +} diff --git a/src/Planner/CollectSets.cpp b/src/Planner/CollectSets.cpp index df3558745fb..8dd7c6637bf 100644 --- a/src/Planner/CollectSets.cpp +++ b/src/Planner/CollectSets.cpp @@ -30,9 +30,8 @@ namespace class CollectSetsVisitor : public ConstInDepthQueryTreeVisitor { public: - explicit CollectSetsVisitor(PlannerContext & planner_context_) //, const SelectQueryOptions & select_query_options_) + explicit CollectSetsVisitor(PlannerContext & planner_context_) : planner_context(planner_context_) - //, select_query_options(select_query_options_) {} void visitImpl(const QueryTreeNodePtr & node) @@ -60,7 +59,7 @@ public: } else if (const auto * constant_node = in_second_argument->as()) { - auto set = makeSetForConstantValue( + auto set = getSetElementsForConstantValue( in_first_argument->getResultType(), constant_node->getValue(), constant_node->getResultType(), @@ -72,18 +71,12 @@ public: set_element_types = left_tuple_type->getElements(); set_element_types = Set::getElementTypes(std::move(set_element_types), settings.transform_null_in); - - // for (auto & element_type : set_element_types) - // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - // element_type = low_cardinality_type->getDictionaryType(); - auto set_key = in_second_argument->getTreeHash(); + if (sets.findTuple(set_key, set_element_types)) return; sets.addFromTuple(set_key, std::move(set), settings); - - //planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set)))); } else if (in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION || @@ -141,14 +134,13 @@ public: private: PlannerContext & planner_context; - //const SelectQueryOptions & select_query_options; }; } -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) //, const SelectQueryOptions & select_query_options) +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context) { - CollectSetsVisitor visitor(planner_context); //, select_query_options); + CollectSetsVisitor visitor(planner_context); visitor.visit(node); } diff --git a/src/Planner/CollectSets.h b/src/Planner/CollectSets.h index e0db802d3b4..5f9f7a5a466 100644 --- a/src/Planner/CollectSets.h +++ b/src/Planner/CollectSets.h @@ -12,6 +12,6 @@ struct SelectQueryOptions; /** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations. * Collected sets are registered in planner context. */ -void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); //, const SelectQueryOptions & select_query_options); +void collectSets(const QueryTreeNodePtr & node, PlannerContext & planner_context); } diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index d524f2bed98..9f6c22f90f3 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -933,8 +933,6 @@ void addBuildSubqueriesForSetsStepIfNeeded( auto subqueries = planner_context->getPreparedSets().getSubqueries(); std::unordered_set useful_sets; - //PreparedSets::SubqueriesForSets subqueries_for_sets; - for (const auto & actions_to_execute : result_actions_to_execute) collectSetsFromActionsDAG(actions_to_execute, useful_sets); @@ -994,7 +992,7 @@ void addAdditionalFilterStepIfNeeded(QueryPlan & query_plan, auto storage = std::make_shared(StorageID{"dummy", "dummy"}, fake_column_descriptions); auto fake_table_expression = std::make_shared(std::move(storage), query_context); - auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, select_query_options, std::move(fake_name_set)); + auto filter_info = buildFilterInfo(additional_result_filter_ast, fake_table_expression, planner_context, std::move(fake_name_set)); if (!filter_info.actions || !query_plan.isInitialized()) return; @@ -1225,7 +1223,7 @@ void Planner::buildPlanForQueryNode() } checkStoragesSupportTransactions(planner_context); - collectSets(query_tree, *planner_context); //, select_query_options); + collectSets(query_tree, *planner_context); collectTableExpressionData(query_tree, planner_context); const auto & settings = query_context->getSettingsRef(); @@ -1524,10 +1522,7 @@ void Planner::buildPlanForQueryNode() } if (!select_query_options.only_analyze) - { - //addCreatingSetsStep(query_plan, planner_context->getPreparedSets().detachSubqueries(planner_context->getQueryContext()), planner_context->getQueryContext()); addBuildSubqueriesForSetsStepIfNeeded(query_plan, select_query_options, planner_context, result_actions_to_execute); - } } SelectQueryInfo Planner::buildSelectQueryInfo() const diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 40b6ec109dc..7575828e64d 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -633,7 +633,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma DataTypes set_element_types; auto in_second_argument_node_type = in_second_argument->getNodeType(); - // std::cerr << "=========== " << in_second_argument->getNodeTypeName() << std::endl; + bool subquery_or_table = in_second_argument_node_type == QueryTreeNodeType::QUERY || in_second_argument_node_type == QueryTreeNodeType::UNION || @@ -649,10 +649,6 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma if (left_tuple_type && left_tuple_type->getElements().size() != 1) set_element_types = left_tuple_type->getElements(); - // for (auto & element_type : set_element_types) - // if (const auto * low_cardinality_type = typeid_cast(element_type.get())) - // element_type = low_cardinality_type->getDictionaryType(); - set_element_types = Set::getElementTypes(std::move(set_element_types), planner_context->getQueryContext()->getSettingsRef().transform_null_in); set = planner_context->getPreparedSets().findTuple(set_key, set_element_types); } diff --git a/src/Planner/PlannerContext.cpp b/src/Planner/PlannerContext.cpp index c8aeebc17c1..3c75d4fbea8 100644 --- a/src/Planner/PlannerContext.cpp +++ b/src/Planner/PlannerContext.cpp @@ -118,50 +118,4 @@ PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second); } -// void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set) -// { -// if (!planner_set.getSet().isValid()) -// throw Exception(ErrorCodes::LOGICAL_ERROR, "Set must be initialized"); - -// const auto & subquery_node = planner_set.getSubqueryNode(); -// if (subquery_node) -// { -// auto node_type = subquery_node->getNodeType(); - -// if (node_type != QueryTreeNodeType::QUERY && -// node_type != QueryTreeNodeType::UNION && -// node_type != QueryTreeNodeType::TABLE) -// throw Exception(ErrorCodes::LOGICAL_ERROR, -// "Invalid node for set table expression. Expected query or union. Actual {}", -// subquery_node->formatASTForErrorMessage()); -// } - -// set_key_to_set.emplace(key, std::move(planner_set)); -// } - -// bool PlannerContext::hasSet(const SetKey & key) const -// { -// return set_key_to_set.contains(key); -// } - -// const PlannerSet & PlannerContext::getSetOrThrow(const SetKey & key) const -// { -// auto it = set_key_to_set.find(key); -// if (it == set_key_to_set.end()) -// throw Exception(ErrorCodes::LOGICAL_ERROR, -// "No set is registered for key {}", -// key); - -// return it->second; -// } - -// PlannerSet * PlannerContext::getSetOrNull(const SetKey & key) -// { -// auto it = set_key_to_set.find(key); -// if (it == set_key_to_set.end()) -// return nullptr; - -// return &it->second; -// } - } diff --git a/src/Planner/PlannerContext.h b/src/Planner/PlannerContext.h index 78c7f8feab5..aceb313d4b5 100644 --- a/src/Planner/PlannerContext.h +++ b/src/Planner/PlannerContext.h @@ -44,56 +44,6 @@ private: using GlobalPlannerContextPtr = std::shared_ptr; -/** PlannerSet is wrapper around Set that is used during query planning. - * - * If subquery node is null, such set is already prepared for execution. - * - * If subquery node is not null, then set must be build from the result of the subquery. - * If subquery node is not null, it must have QUERY or UNION type. - */ -class PlannerSet -{ - -}; - -// { -// public: -// /// Construct planner set that is ready for execution -// explicit PlannerSet(FutureSetPtr set_) -// : set(std::move(set_)) -// {} - -// /// Construct planner set with set and subquery node -// explicit PlannerSet(QueryTreeNodePtr subquery_node_) -// //: set(promise_to_build_set.get_future()) -// : subquery_node(std::move(subquery_node_)) -// {} - -// /// Get a reference to a set that might be not built yet -// const FutureSetPtr & getSet() const -// { -// return set; -// } - -// /// Get subquery node -// const QueryTreeNodePtr & getSubqueryNode() const -// { -// return subquery_node; -// } - -// /// This promise will be fulfilled when set is built and all FutureSet objects will become ready -// // std::promise extractPromiseToBuildSet() -// // { -// // return std::move(promise_to_build_set); -// // } - -// private: -// //std::promise promise_to_build_set; -// FutureSetPtr set; - -// QueryTreeNodePtr subquery_node; -// }; - class PlannerContext { public: @@ -181,29 +131,9 @@ public: using SetKey = std::string; - // using SetKeyToSet = std::unordered_map; - - // /// Create set key for set source node + /// Create set key for set source node static SetKey createSetKey(const QueryTreeNodePtr & set_source_node); - // /// Register set for set key - // void registerSet(const SetKey & key, PlannerSet planner_set); - - // /// Returns true if set is registered for key, false otherwise - // bool hasSet(const SetKey & key) const; - - // /// Get set for key, if no set is registered logical exception is thrown - // const PlannerSet & getSetOrThrow(const SetKey & key) const; - - // /// Get set for key, if no set is registered null is returned - // PlannerSet * getSetOrNull(const SetKey & key); - - // /// Get registered sets - // const SetKeyToSet & getRegisteredSets() const - // { - // return set_key_to_set; - // } - PreparedSets & getPreparedSets() { return prepared_sets; } private: diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index c27ee72437c..5d8f8ca8741 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -403,8 +403,7 @@ void updatePrewhereOutputsIfNeeded(SelectQueryInfo & table_expression_query_info FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options) + PlannerContextPtr & planner_context) { auto storage_id = storage->getStorageID(); const auto & query_context = planner_context->getQueryContext(); @@ -413,13 +412,12 @@ FilterDAGInfo buildRowPolicyFilterIfNeeded(const StoragePtr & storage, if (!row_policy_filter) return {}; - return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context, select_query_options); + return buildFilterInfo(row_policy_filter->expression, table_expression_query_info.table_expression, planner_context); } FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options) + PlannerContextPtr & planner_context) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -445,15 +443,14 @@ FilterDAGInfo buildCustomKeyFilterIfNeeded(const StoragePtr & storage, *storage, query_context); - return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context, select_query_options); + return buildFilterInfo(parallel_replicas_custom_filter_ast, table_expression_query_info.table_expression, planner_context); } /// Apply filters from additional_table_filters setting FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, const String & table_expression_alias, SelectQueryInfo & table_expression_query_info, - PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options) + PlannerContextPtr & planner_context) { const auto & query_context = planner_context->getQueryContext(); const auto & settings = query_context->getSettingsRef(); @@ -487,7 +484,7 @@ FilterDAGInfo buildAdditionalFiltersIfNeeded(const StoragePtr & storage, return {}; table_expression_query_info.additional_filter_ast = additional_filter_ast; - return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context, select_query_options); + return buildFilterInfo(additional_filter_ast, table_expression_query_info.table_expression, planner_context); } JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, @@ -697,14 +694,14 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } }; - auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context, select_query_options); + auto row_policy_filter_info = buildRowPolicyFilterIfNeeded(storage, table_expression_query_info, planner_context); add_filter(row_policy_filter_info, "Row-level security filter"); if (query_context->getParallelReplicasMode() == Context::ParallelReplicasMode::CUSTOM_KEY) { if (settings.parallel_replicas_count > 1) { - auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context, select_query_options); + auto parallel_replicas_custom_key_filter_info = buildCustomKeyFilterIfNeeded(storage, table_expression_query_info, planner_context); add_filter(parallel_replicas_custom_key_filter_info, "Parallel replicas custom key filter"); } else @@ -719,7 +716,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres } const auto & table_expression_alias = table_expression->getAlias(); - auto additional_filters_info = buildAdditionalFiltersIfNeeded(storage, table_expression_alias, table_expression_query_info, planner_context, select_query_options); + 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); diff --git a/src/Planner/Utils.cpp b/src/Planner/Utils.cpp index 403e3f03997..733db0f00bc 100644 --- a/src/Planner/Utils.cpp +++ b/src/Planner/Utils.cpp @@ -429,7 +429,6 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, - [[maybe_unused]] const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter) { const auto & query_context = planner_context->getQueryContext(); @@ -447,7 +446,7 @@ FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, } collectSourceColumns(filter_query_tree, planner_context); - collectSets(filter_query_tree, *planner_context); //, select_query_options); + collectSets(filter_query_tree, *planner_context); auto filter_actions_dag = std::make_shared(); diff --git a/src/Planner/Utils.h b/src/Planner/Utils.h index 8071e201f88..d9412800e61 100644 --- a/src/Planner/Utils.h +++ b/src/Planner/Utils.h @@ -82,7 +82,6 @@ SelectQueryInfo buildSelectQueryInfo(const QueryTreeNodePtr & query_tree, const FilterDAGInfo buildFilterInfo(ASTPtr filter_expression, const QueryTreeNodePtr & table_expression, PlannerContextPtr & planner_context, - const SelectQueryOptions & select_query_options, NameSet table_expression_required_names_without_filter = {}); ASTPtr parseAdditionalResultFilter(const Settings & settings); diff --git a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp index ebd58c3dc95..c54d32c1385 100644 --- a/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp +++ b/src/Processors/QueryPlan/CreateSetAndFilterOnTheFlyStep.cpp @@ -105,7 +105,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep( : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , column_names(column_names_) , max_rows_in_set(max_rows_in_set_) - , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, 0, true)) + , own_set(std::make_shared(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), 0, true)) , filtering_set(nullptr) , crosswise_connection(crosswise_connection_) , position(position_) diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index aae632787b9..3e4dfb0c7d1 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -124,7 +124,7 @@ void CreatingSetsStep::describePipeline(FormatSettings & settings) const IQueryPlanStep::describePipeline(processors, settings); } -void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subquery, ContextPtr context) +void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subqueries, ContextPtr context) { DataStreams input_streams; input_streams.emplace_back(query_plan.getCurrentDataStream()); @@ -133,7 +133,7 @@ void addCreatingSetsStep(QueryPlan & query_plan, std::vector(std::move(query_plan))); query_plan = QueryPlan(); - for (auto & future_set : sets_from_subquery) + for (auto & future_set : subqueries) { if (future_set->get()) continue; @@ -157,18 +157,11 @@ void addCreatingSetsStep(QueryPlan & query_plan, std::vector> DelayedCreatingSetsStep::makePlansForSets(DelayedCreatingSetsStep && step) { - // DataStreams input_streams; - // input_streams.emplace_back(query_plan.getCurrentDataStream()); - std::vector> plans; - // plans.emplace_back(std::make_unique(std::move(query_plan))); - // query_plan = QueryPlan(); - for (auto & future_set : step.sets_from_subquery) + for (auto & future_set : step.subqueries) { if (future_set->get()) continue; @@ -179,7 +172,6 @@ std::vector> DelayedCreatingSetsStep::makePlansForSet plan->optimize(QueryPlanOptimizationSettings::fromContext(step.context)); - //input_streams.emplace_back(plan->getCurrentDataStream()); plans.emplace_back(std::move(plan)); } @@ -199,8 +191,8 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, } DelayedCreatingSetsStep::DelayedCreatingSetsStep( - DataStream input_stream, std::vector> sets_from_subquery_, ContextPtr context_) - : sets_from_subquery(std::move(sets_from_subquery_)), context(std::move(context_)) + DataStream input_stream, PreparedSets::Subqueries subqueries_, ContextPtr context_) + : subqueries(std::move(subqueries_)), context(std::move(context_)) { input_streams = {input_stream}; output_stream = std::move(input_stream); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 7110775da79..a90b70a2fa4 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -47,10 +47,12 @@ public: void describePipeline(FormatSettings & settings) const override; }; +/// This is a temporary step which is converted to CreatingSetStep after plan optimization. +/// Can't be used by itself. class DelayedCreatingSetsStep final : public IQueryPlanStep { public: - DelayedCreatingSetsStep(DataStream input_stream, std::vector> sets_from_subquery_, ContextPtr context_); + DelayedCreatingSetsStep(DataStream input_stream, PreparedSets::Subqueries subqueries_, ContextPtr context_); String getName() const override { return "DelayedCreatingSets"; } @@ -59,14 +61,14 @@ public: static std::vector> makePlansForSets(DelayedCreatingSetsStep && step); ContextPtr getContext() const { return context; } - std::vector> detachSets() { return std::move(sets_from_subquery); } + PreparedSets::Subqueries detachSets() { return std::move(subqueries); } private: - std::vector> sets_from_subquery; + PreparedSets::Subqueries subqueries; ContextPtr context; }; -void addCreatingSetsStep(QueryPlan & query_plan, std::vector> sets_from_subquery, ContextPtr context); +void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subqueries, ContextPtr context); void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context); diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 8e2221d564d..9b9cc221ca8 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -43,7 +43,6 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, - [[maybe_unused]] PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, @@ -99,7 +98,7 @@ std::unique_ptr createLocalPlan( } else { - auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); //, prepared_sets); + auto interpreter = InterpreterSelectQuery(query_ast, new_context, select_query_options); interpreter.buildQueryPlan(*query_plan); } diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h index cf59027a33f..c08b9bdf67e 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h @@ -18,7 +18,6 @@ std::unique_ptr createLocalPlan( const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, - PreparedSetsPtr prepared_sets, size_t shard_num, size_t shard_count, size_t replica_num, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e549a06877e..1e21d13e2b1 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1220,13 +1220,8 @@ static void buildIndexes( const auto & partition_key = metadata_snapshot->getPartitionKey(); auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); auto minmax_expression_actions = data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context)); - // minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); - // if (context->getSettingsRef().allow_experimental_analyzer) indexes->minmax_idx_condition.emplace(filter_actions_dag, context, minmax_columns_names, minmax_expression_actions, NameSet()); - // else - // indexes->minmax_idx_condition.emplace(query_info, context, minmax_columns_names, minmax_expression_actions); - indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */); } @@ -1402,32 +1397,10 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl( result.column_names_to_read.push_back(ExpressionActions::getSmallestColumn(available_real_columns).name); } - // storage_snapshot->check(result.column_names_to_read); - // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); const Names & primary_key_column_names = primary_key.column_names; - // if (!key_condition) - // { - // if (settings.query_plan_optimize_primary_key) - // { - // NameSet array_join_name_set; - // if (query_info.syntax_analyzer_result) - // array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); - - // key_condition.emplace(query_info.filter_actions_dag, - // context, - // primary_key_column_names, - // primary_key.expression, - // array_join_name_set); - // } - // else - // { - // key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression); - // } - // } - if (!indexes) buildIndexes(indexes, query_info.filter_actions_dag, data, context, query_info, metadata_snapshot); diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 16cb06a94d6..ed740e3e242 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -97,7 +97,6 @@ ReadFromRemote::ReadFromRemote( ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, - PreparedSetsPtr prepared_sets_, StorageID main_table_, ASTPtr table_func_ptr_, ContextMutablePtr context_, @@ -110,7 +109,6 @@ ReadFromRemote::ReadFromRemote( : ISourceStep(DataStream{.header = std::move(header_)}) , shards(std::move(shards_)) , stage(stage_) - , prepared_sets(std::move(prepared_sets_)) , main_table(std::move(main_table_)) , table_func_ptr(std::move(table_func_ptr_)) , context(std::move(context_)) @@ -152,7 +150,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream my_context = context, my_throttler = throttler, my_main_table = main_table, my_table_func_ptr = table_func_ptr, my_scalars = scalars, my_external_tables = external_tables, - my_stage = stage, my_prepared_sets = prepared_sets, local_delay = shard.local_delay, + my_stage = stage, local_delay = shard.local_delay, add_agg_info, add_totals, add_extremes, async_read, async_query_sending]() mutable -> QueryPipelineBuilder { @@ -187,7 +185,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStream if (try_results.empty() || local_delay < max_remote_delay) { auto plan = createLocalPlan( - query, header, my_context, my_stage, my_prepared_sets, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); + query, header, my_context, my_stage, my_shard.shard_info.shard_num, my_shard_count, 0, 0, /*coordinator=*/nullptr); return std::move(*plan->buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(my_context), diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 405b5727ff2..d4005d81f1b 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -26,7 +26,6 @@ public: ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, - PreparedSetsPtr prepared_sets_, StorageID main_table_, ASTPtr table_func_ptr_, ContextMutablePtr context_, @@ -47,7 +46,6 @@ public: private: ClusterProxy::SelectStreamFactory::Shards shards; QueryProcessingStage::Enum stage; - PreparedSetsPtr prepared_sets; StorageID main_table; ASTPtr table_func_ptr; ContextMutablePtr context; diff --git a/src/Storages/KVStorageUtils.cpp b/src/Storages/KVStorageUtils.cpp index 4c1fe4fc559..16ab99d03b4 100644 --- a/src/Storages/KVStorageUtils.cpp +++ b/src/Storages/KVStorageUtils.cpp @@ -77,7 +77,6 @@ bool traverseASTFilter( if (!future_set) return false; - //if (!future_set->isReady()) future_set->buildOrderedSetInplace(context); auto set = future_set->get(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index d41dd30098b..f909d854cf6 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1194,8 +1194,6 @@ bool KeyCondition::tryPrepareSetIndex( RPNElement & out, size_t & out_key_column_num) { - // std::cerr << "::: tryPrepareSetIndex for " << func.getColumnName() << std::endl; - // std::cerr << StackTrace().toString() << std::endl; const auto & left_arg = func.getArgumentAt(0); out_key_column_num = 0; @@ -1239,56 +1237,27 @@ bool KeyCondition::tryPrepareSetIndex( } if (indexes_mapping.empty()) - { - // std::cerr << ".. index mapping is empty\n"; return false; - } const auto right_arg = func.getArgumentAt(1); - // LOG_TRACE(&Poco::Logger::get("KK"), "Trying to get set for {}", right_arg.getColumnName()); - auto future_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); if (!future_set) - { - // std::cerr << ".. no future set\n"; return false; - } - // LOG_TRACE(&Poco::Logger::get("KK"), "Found set for {}", right_arg.getColumnName()); - - //if (!future_set->isReady()) - //{ - // LOG_TRACE(&Poco::Logger::get("KK"), "Building set inplace for {}", right_arg.getColumnName()); - future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); - //} - - auto prepared_set = future_set->get(); + auto prepared_set = future_set->buildOrderedSetInplace(right_arg.getTreeContext().getQueryContext()); if (!prepared_set) - { - - // std::cerr << ".. no prepared set\n"; return false; - } - - // LOG_TRACE(&Poco::Logger::get("KK"), "Set if ready for {}", right_arg.getColumnName()); /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) - { - - // std::cerr << ".. no explicit elements\n"; return false; - } - - // LOG_TRACE(&Poco::Logger::get("KK"), "Has explicit elements for {}", right_arg.getColumnName()); prepared_set->checkColumnsNumber(left_args_count); for (size_t i = 0; i < indexes_mapping.size(); ++i) prepared_set->checkTypesEqual(indexes_mapping[i].tuple_index, data_types[i]); out.set_index = std::make_shared(prepared_set->getSetElements(), std::move(indexes_mapping)); - // std::cerr << ".. can use\n"; return true; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 125fe98203f..ee515106591 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1839,9 +1839,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( if (partition_pruner) { - auto val = partition_pruner->canBePruned(*part); - // std::cerr << "... part " << part->getNameWithState() << " cbp ? " << val << std::endl; - if (val) + if (partition_pruner->canBePruned(*part)) continue; } diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index e2bf9bde674..398a85e92ac 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -31,21 +31,13 @@ namespace ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set) { - // std::cerr << "====== " << prepared_set->getDataTypes().size() << std::endl; if (prepared_set->getDataTypes().size() == 1) return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; Columns set_elements; for (auto & set_element : prepared_set->getSetElements()) - { - // std::cerr << set_element->dumpStructure() << std::endl; - set_elements.emplace_back(set_element->convertToFullColumnIfConst()); - } - // for (auto & set_element : prepared_set->getElementsTypes()) - // { - // // std::cerr << set_element->getName() << std::endl; - // } + set_elements.emplace_back(set_element->convertToFullColumnIfConst()); return {ColumnTuple::create(set_elements), std::make_shared(prepared_set->getElementsTypes()), "dummy"}; } @@ -319,30 +311,17 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo if (functionIsInOrGlobalInOperator(function_name)) { - //std::cerr << StackTrace().toString() << std::endl; - - auto future_set = rhs_argument.tryGetPreparedSet(); - - //std::cerr << "==== Finding set for MergeTreeBF " << bool(future_set) << std::endl; - - if (future_set) // && !future_set->isReady()) + if (auto future_set = rhs_argument.tryGetPreparedSet(); future_set) { - //std::cerr << "==== not ready, building " << std::endl; - future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); - } - - ConstSetPtr prepared_set; - if (future_set) - prepared_set = future_set->get(); - - //std::cerr << "==== Prep set for MergeTreeBF " << bool(prepared_set) << ' ' << (prepared_set ? prepared_set->hasExplicitSetElements() : false) << std::endl; - - if (prepared_set && prepared_set->hasExplicitSetElements()) - { - const auto prepared_info = getPreparedSetInfo(prepared_set); - // std::cerr << "...... " << prepared_info.dumpStructure() << std::endl; - if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) - maybe_useful = true; + if (auto prepared_set = future_set->buildOrderedSetInplace(rhs_argument.getTreeContext().getQueryContext()); prepared_set) + { + if (prepared_set->hasExplicitSetElements()) + { + const auto prepared_info = getPreparedSetInfo(prepared_set); + if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) + maybe_useful = true; + } + } } } else if (function_name == "equals" || @@ -387,7 +366,6 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( size_t row_size = column->size(); size_t position = header.getPositionByName(key_node_column_name); const DataTypePtr & index_type = header.getByPosition(position).type; - // std::cerr << "::::: " << ColumnWithTypeAndName{column, type, ""}.dumpStructure() << " -> " << index_type->getName() << std::endl; const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 422f504d046..b6a2cafe245 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -625,13 +625,10 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( return false; auto future_set = right_argument.tryGetPreparedSet(data_types); - if (future_set) // && !future_set->isReady()) - future_set->buildOrderedSetInplace(right_argument.getTreeContext().getQueryContext()); - - ConstSetPtr prepared_set; - if (future_set) - prepared_set = future_set->get(); + if (!future_set) + return false; + auto prepared_set = future_set->buildOrderedSetInplace(right_argument.getTreeContext().getQueryContext()); if (!prepared_set || !prepared_set->hasExplicitSetElements()) return false; diff --git a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp index 2814b8521f6..6b4919c545d 100644 --- a/src/Storages/MergeTree/MergeTreeIndexInverted.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexInverted.cpp @@ -655,19 +655,11 @@ bool MergeTreeConditionInverted::tryPrepareSetGinFilter( if (key_tuple_mapping.empty()) return false; - //std::cerr << "==== Finding set for MergeTreeConditionInverted\n"; - auto future_set = rhs.tryGetPreparedSet(); + if (!future_set) + return false; - //std::cerr << "==== Set for MergeTreeConditionInverted" << bool(future_set) << std::endl; - - if (future_set) // && !future_set->isReady()) - future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); - - ConstSetPtr prepared_set; - if (future_set) - prepared_set = future_set->get(); - + auto prepared_set = future_set->buildOrderedSetInplace(rhs.getTreeContext().getQueryContext()); if (!prepared_set || !prepared_set->hasExplicitSetElements()) return false; diff --git a/src/Storages/MergeTree/PartitionPruner.cpp b/src/Storages/MergeTree/PartitionPruner.cpp index a397a1475d1..97bb9f3b4d4 100644 --- a/src/Storages/MergeTree/PartitionPruner.cpp +++ b/src/Storages/MergeTree/PartitionPruner.cpp @@ -29,12 +29,6 @@ PartitionPruner::PartitionPruner(const StorageMetadataPtr & metadata, ActionsDAG , partition_condition(filter_actions_dag, context, partition_key.column_names, partition_key.expression, {}, true /* single_point */, strict) , useless(strict ? partition_condition.anyUnknownOrAlwaysTrue() : partition_condition.alwaysUnknownOrTrue()) { - // auto description = getKeyCondition().getDescription(); - // std::cerr << ".... " << description.condition << std::endl; - // std::cerr << filter_actions_dag->dumpDAG() << std::endl; - // for (const auto & name : partition_key.column_names) - // std::cerr << ". " << name << std::endl; - // std::cerr << partition_key.expression->dumpActions() << std::endl; } bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) @@ -52,8 +46,6 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) else { const auto & partition_value = part.partition.value; - // for (const auto & val : partition_value) - // std::cerr << val.dump() << std::endl; std::vector index_value(partition_value.begin(), partition_value.end()); for (auto & field : index_value) { @@ -64,7 +56,6 @@ bool PartitionPruner::canBePruned(const IMergeTreeDataPart & part) is_valid = partition_condition.mayBeTrueInRange( partition_value.size(), index_value.data(), index_value.data(), partition_key.data_types); - // std::cerr << "may be true " << is_valid << std::endl; partition_filter_map.emplace(partition_id, is_valid); if (!is_valid) diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a69db7d80c0..c46192ab43b 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -88,10 +88,8 @@ #include #include #include -#include #include #include -#include #include #include #include @@ -769,8 +767,7 @@ void StorageDistributed::read( header, snapshot_data.objects_by_shard, storage_snapshot, - processed_stage, - query_info.prepared_sets); + processed_stage); auto settings = local_context->getSettingsRef(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 655b3d5a8f4..c02c96f62be 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -231,8 +231,7 @@ void StorageMergeTree::read( header, {}, storage_snapshot, - processed_stage, - query_info.prepared_sets); + processed_stage); ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 01f663aeee4..bb99e21e4ab 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4866,8 +4866,7 @@ void StorageReplicatedMergeTree::read( header, {}, storage_snapshot, - processed_stage, - query_info.prepared_sets); + processed_stage); ClusterProxy::executeQueryWithParallelReplicas( query_plan, getStorageID(), /*remove_table_function_ptr*/ nullptr, diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp index f9fdcdf02a6..79369ab4bcb 100644 --- a/src/Storages/StorageSet.cpp +++ b/src/Storages/StorageSet.cpp @@ -147,7 +147,7 @@ StorageSet::StorageSet( const String & comment, bool persistent_) : StorageSetOrJoinBase{disk_, relative_path_, table_id_, columns_, constraints_, comment, persistent_} - , set(std::make_shared(SizeLimits(), false, 0, true)) + , set(std::make_shared(SizeLimits(), 0, true)) { Block header = getInMemoryMetadataPtr()->getSampleBlock(); set->setHeader(header.getColumnsWithTypeAndName()); @@ -176,7 +176,7 @@ void StorageSet::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn Block header = metadata_snapshot->getSampleBlock(); increment = 0; - set = std::make_shared(SizeLimits(), false, 0, true); + set = std::make_shared(SizeLimits(), 0, true); set->setHeader(header.getColumnsWithTypeAndName()); } diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 4cb08b90c2a..d361a4173c1 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -323,14 +323,8 @@ static void extractPathImpl(const ActionsDAG::Node & node, Paths & res, ContextP if (!future_set) return; - //if (!future_set->isReady()) - future_set->buildOrderedSetInplace(context); - - auto set = future_set->get(); - if (!set) - return; - - if (!set->hasExplicitSetElements()) + auto set = future_set->buildOrderedSetInplace(context); + if (!set || !set->hasExplicitSetElements()) return; set->checkColumnsNumber(1); diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index b83afe766aa..907fc0cd22c 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -89,25 +89,6 @@ ASTPtr buildWhereExpression(const ASTs & functions) return makeASTFunction("and", functions); } -// void buildSets(const ASTPtr & expression, ExpressionAnalyzer & analyzer) -// { -// const auto * func = expression->as(); -// if (func && functionIsInOrGlobalInOperator(func->name)) -// { -// const IAST & args = *func->arguments; -// const ASTPtr & arg = args.children.at(1); -// if (arg->as() || arg->as()) -// { -// analyzer.tryMakeSetForIndexFromSubquery(arg); -// } -// } -// else -// { -// for (const auto & child : expression->children) -// buildSets(child, analyzer); -// } -// } - } namespace VirtualColumnUtils @@ -208,7 +189,6 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, ContextPtr contex /// Let's analyze and calculate the prepared expression. auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(expression_ast, syntax_result, context); - //buildSets(expression_ast, analyzer); ExpressionActionsPtr actions = analyzer.getActions(false /* add alises */, true /* project result */, CompileExpressions::yes); for (const auto & node : actions->getNodes()) diff --git a/src/Storages/buildQueryTreeForShard.cpp b/src/Storages/buildQueryTreeForShard.cpp index fa4730cbe84..1ee7d747fcc 100644 --- a/src/Storages/buildQueryTreeForShard.cpp +++ b/src/Storages/buildQueryTreeForShard.cpp @@ -1,3 +1,4 @@ + #include #include @@ -15,6 +16,8 @@ #include #include #include +#include +#include namespace DB { @@ -276,11 +279,14 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, temporary_table_expression_node->setTemporaryTableName(temporary_table_name); auto table_out = external_storage->write({}, external_storage->getInMemoryMetadataPtr(), mutable_context, /*async_insert=*/false); - auto io = interpreter.execute(); - io.pipeline.complete(std::move(table_out)); - CompletedPipelineExecutor executor(io.pipeline); - executor.execute(); + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(mutable_context); + auto build_pipeline_settings = BuildQueryPipelineSettings::fromContext(mutable_context); + auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*query_plan.buildQueryPipeline(optimization_settings, build_pipeline_settings))); + + pipeline.complete(std::move(table_out)); + CompletedPipelineExecutor executor(pipeline); + executor.execute(); mutable_context->addExternalTable(temporary_table_name, std::move(external_storage_holder)); return temporary_table_expression_node; @@ -291,14 +297,13 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node, QueryTreeNodePtr buildQueryTreeForShard(SelectQueryInfo & query_info, QueryTreeNodePtr query_tree_to_modify) { auto & planner_context = query_info.planner_context; - const auto & query_context = planner_context->getQueryContext(); CollectColumnSourceToColumnsVisitor collect_column_source_to_columns_visitor; collect_column_source_to_columns_visitor.visit(query_tree_to_modify); const auto & column_source_to_columns = collect_column_source_to_columns_visitor.getColumnSourceToColumns(); - DistributedProductModeRewriteInJoinVisitor visitor(query_info.planner_context->getQueryContext()); + DistributedProductModeRewriteInJoinVisitor visitor(planner_context->getQueryContext()); visitor.visit(query_tree_to_modify); auto replacement_map = visitor.getReplacementMap(); From 376b65ee1efeeec438d52750e57182aef7379cad Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 22 Jun 2023 19:06:49 +0200 Subject: [PATCH 084/171] Define Thrift version for parquet --- contrib/arrow-cmake/CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 5fe942d1cd0..836452c53ed 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -514,6 +514,10 @@ if (SANITIZE STREQUAL "undefined") target_compile_options(_arrow PRIVATE -fno-sanitize=undefined) endif () +# Define Thrift version for parquet (we use 0.16.0) +add_definitions(-DPARQUET_THRIFT_VERSION_MAJOR=0) +add_definitions(-DPARQUET_THRIFT_VERSION_MINOR=16) + # === tools set(TOOLS_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/tools/parquet") From 1d379108a314abc9744069c4e697b9e87818fa7b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 22 Jun 2023 19:23:14 +0200 Subject: [PATCH 085/171] Update arrow version in cmake --- contrib/arrow-cmake/CMakeLists.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 836452c53ed..46b86cb4ddb 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -31,12 +31,12 @@ endif() set (CMAKE_CXX_STANDARD 17) -set(ARROW_VERSION "6.0.1") +set(ARROW_VERSION "11.0.0") string(REGEX MATCH "^[0-9]+\\.[0-9]+\\.[0-9]+" ARROW_BASE_VERSION "${ARROW_VERSION}") -set(ARROW_VERSION_MAJOR "6") +set(ARROW_VERSION_MAJOR "11") set(ARROW_VERSION_MINOR "0") -set(ARROW_VERSION_PATCH "1") +set(ARROW_VERSION_PATCH "0") if(ARROW_VERSION_MAJOR STREQUAL "0") # Arrow 0.x.y => SO version is "x", full SO version is "x.y.0" From ba4b5c335047f72bbed063434b5f4b7a38459fe6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Jun 2023 18:15:16 +0000 Subject: [PATCH 086/171] Fix stupid bug. --- src/Interpreters/PreparedSets.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/PreparedSets.cpp b/src/Interpreters/PreparedSets.cpp index 428ef873bc5..67822ecf440 100644 --- a/src/Interpreters/PreparedSets.cpp +++ b/src/Interpreters/PreparedSets.cpp @@ -230,12 +230,11 @@ FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Set const auto & set_types = from_tuple->getTypes(); auto & sets_by_hash = sets_from_tuple[key]; - auto types = from_tuple->getTypes(); for (const auto & set : sets_by_hash) if (equals(set->getTypes(), set_types)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, from_tuple->getTypes())); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, set_types)); - sets_by_hash.push_back(std::move(from_tuple)); + sets_by_hash.push_back(from_tuple); return from_tuple; } From 75391afbd82b3c2aa5caae8fed82324334a49bbb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Jun 2023 18:16:48 +0000 Subject: [PATCH 087/171] Fix typo. --- src/Interpreters/ActionsVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index bf78868463a..cfbe53b5e4d 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1406,7 +1406,7 @@ FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool /// /// Mutation is executed in two stages: /// * first, query 'SELECT count() FROM table WHERE ...' is executed to get the set of affected parts (using analyzer) - /// * second, every part is mutated separately, where plan is build "manually", usign this code as well + /// * second, every part is mutated separately, where plan is build "manually", using this code as well /// To share the Set in between first and second stage, we should use the same hash. /// New analyzer is uses a hash from query tree, so here we also build a query tree. /// From f226397fa35bc7637a1a2a078018e760ecb09d6b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Jun 2023 19:21:08 +0000 Subject: [PATCH 088/171] Fix check. --- src/Interpreters/Set.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index b42ff102f72..b8b61c7c11f 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -173,9 +173,6 @@ void Set::setHeader(const ColumnsWithTypeAndName & header) void Set::fillSetElements() { - if (data.getTotalRowCount()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot convert set to ordered because it is not empty"); - fill_set_elements = true; set_elements.reserve(keys_size); for (const auto & type : set_elements_types) From db8120722f5281996e29972ba2b9f3eab965e6d1 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 23 Jun 2023 12:00:20 +0200 Subject: [PATCH 089/171] Separated functionality of Iterator into GlobIterator and KeysIterator, added test for filter by file --- src/Storages/StorageAzureBlob.cpp | 332 +++++++++--------- src/Storages/StorageAzureBlob.h | 64 +++- .../test_storage_azure_blob_storage/test.py | 16 + 3 files changed, 233 insertions(+), 179 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 91dc92f09e8..8a1c81e808e 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -611,19 +611,19 @@ Pipe StorageAzureBlob::read( requested_virtual_columns.push_back(virtual_column); } - std::shared_ptr iterator_wrapper; + std::shared_ptr iterator_wrapper; if (configuration.withGlobs()) { /// Iterate through disclosed globs and make a source for each file - iterator_wrapper = std::make_shared( - object_storage.get(), configuration.container, std::nullopt, - configuration.blob_path, query_info.query, virtual_block, local_context, nullptr); + iterator_wrapper = std::make_shared( + object_storage.get(), configuration.container, configuration.blob_path, + query_info.query, virtual_block, local_context, nullptr); } else { - iterator_wrapper = std::make_shared( + iterator_wrapper = std::make_shared( object_storage.get(), configuration.container, configuration.blobs_paths, - std::nullopt, query_info.query, virtual_block, local_context, nullptr); + query_info.query, virtual_block, local_context, nullptr); } ColumnsDescription columns_description; @@ -786,201 +786,129 @@ static void addPathToVirtualColumns(Block & block, const String & path, size_t i block.getByName("_idx").column->assumeMutableRef().insert(idx); } -StorageAzureBlobSource::Iterator::Iterator( +StorageAzureBlobSource::GlobIterator::GlobIterator( AzureObjectStorage * object_storage_, const std::string & container_, - std::optional keys_, - std::optional blob_path_with_globs_, + String blob_path_with_globs_, ASTPtr query_, const Block & virtual_header_, ContextPtr context_, RelativePathsWithMetadata * outer_blobs_) - : WithContext(context_) + : IIterator(context_) , object_storage(object_storage_) , container(container_) - , keys(keys_) , blob_path_with_globs(blob_path_with_globs_) , query(query_) , virtual_header(virtual_header_) , outer_blobs(outer_blobs_) { - if (keys.has_value() && blob_path_with_globs.has_value()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot specify keys and glob simultaneously it's a bug"); - if (!keys.has_value() && !blob_path_with_globs.has_value()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Both keys and glob mask are not specified"); + const String key_prefix = blob_path_with_globs.substr(0, blob_path_with_globs.find_first_of("*?{")); - if (keys) + /// We don't have to list bucket, because there is no asterisks. + if (key_prefix.size() == blob_path_with_globs.size()) { - Strings all_keys = *keys; - - blobs_with_metadata.emplace(); - /// Create a virtual block with one row to construct filter - if (query && virtual_header && !all_keys.empty()) - { - /// Append "idx" column as the filter result - virtual_header.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); - - auto block = virtual_header.cloneEmpty(); - addPathToVirtualColumns(block, fs::path(container) / all_keys.front(), 0); - - VirtualColumnUtils::prepareFilterBlockWithQuery(query, getContext(), block, filter_ast); - - if (filter_ast) - { - block = virtual_header.cloneEmpty(); - for (size_t i = 0; i < all_keys.size(); ++i) - addPathToVirtualColumns(block, fs::path(container) / all_keys[i], i); - - VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); - const auto & idxs = typeid_cast(*block.getByName("_idx").column); - - Strings filtered_keys; - filtered_keys.reserve(block.rows()); - for (UInt64 idx : idxs.getData()) - filtered_keys.emplace_back(std::move(all_keys[idx])); - - all_keys = std::move(filtered_keys); - } - } - - for (auto && key : all_keys) - { - ObjectMetadata object_metadata = object_storage->getObjectMetadata(key); - total_size += object_metadata.size_bytes; - blobs_with_metadata->emplace_back(RelativePathWithMetadata{key, object_metadata}); - if (outer_blobs) - outer_blobs->emplace_back(blobs_with_metadata->back()); - } - } - else - { - const String key_prefix = blob_path_with_globs->substr(0, blob_path_with_globs->find_first_of("*?{")); - - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == blob_path_with_globs->size()) - { - ObjectMetadata object_metadata = object_storage->getObjectMetadata(*blob_path_with_globs); - blobs_with_metadata->emplace_back(*blob_path_with_globs, object_metadata); - if (outer_blobs) - outer_blobs->emplace_back(blobs_with_metadata->back()); - return; - } - - object_storage_iterator = object_storage->iterate(key_prefix); - - matcher = std::make_unique(makeRegexpPatternFromGlobs(*blob_path_with_globs)); - - if (!matcher->ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", *blob_path_with_globs, matcher->error()); - - recursive = *blob_path_with_globs == "/**" ? true : false; + ObjectMetadata object_metadata = object_storage->getObjectMetadata(blob_path_with_globs); + blobs_with_metadata.emplace_back(blob_path_with_globs, object_metadata); + if (outer_blobs) + outer_blobs->emplace_back(blobs_with_metadata.back()); + return; } + object_storage_iterator = object_storage->iterate(key_prefix); + + matcher = std::make_unique(makeRegexpPatternFromGlobs(blob_path_with_globs)); + + if (!matcher->ok()) + throw Exception( + ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", blob_path_with_globs, matcher->error()); + + recursive = blob_path_with_globs == "/**" ? true : false; } -RelativePathWithMetadata StorageAzureBlobSource::Iterator::next() +RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next() { + std::lock_guard lock(next_mutex); + if (is_finished) return {}; - if (keys) + bool need_new_batch = blobs_with_metadata.empty() || index >= blobs_with_metadata.size(); + + if (need_new_batch) { - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - if (current_index >= blobs_with_metadata->size()) + RelativePathsWithMetadata new_batch; + while (new_batch.empty()) { - is_finished = true; - return {}; - } - - return (*blobs_with_metadata)[current_index]; - } - else - { - bool need_new_batch = false; - { - std::lock_guard lock(next_mutex); - need_new_batch = !blobs_with_metadata || index >= blobs_with_metadata->size(); - } - - if (need_new_batch) - { - RelativePathsWithMetadata new_batch; - while (new_batch.empty()) + auto result = object_storage_iterator->getCurrrentBatchAndScheduleNext(); + if (result.has_value()) { - auto result = object_storage_iterator->getCurrrentBatchAndScheduleNext(); - if (result.has_value()) - { - new_batch = result.value(); - } - else - { - is_finished = true; - return {}; - } - - for (auto it = new_batch.begin(); it != new_batch.end();) - { - if (!recursive && !re2::RE2::FullMatch(it->relative_path, *matcher)) - it = new_batch.erase(it); - else - ++it; - } - } - - index.store(0, std::memory_order_relaxed); - if (!is_initialized) - { - createFilterAST(new_batch.front().relative_path); - is_initialized = true; - } - - if (filter_ast) - { - auto block = virtual_header.cloneEmpty(); - for (size_t i = 0; i < new_batch.size(); ++i) - addPathToVirtualColumns(block, fs::path(container) / new_batch[i].relative_path, i); - - VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); - const auto & idxs = typeid_cast(*block.getByName("_idx").column); - - std::lock_guard lock(next_mutex); - blob_path_with_globs.reset(); - blob_path_with_globs.emplace(); - for (UInt64 idx : idxs.getData()) - { - total_size.fetch_add(new_batch[idx].metadata.size_bytes, std::memory_order_relaxed); - blobs_with_metadata->emplace_back(std::move(new_batch[idx])); - if (outer_blobs) - outer_blobs->emplace_back(blobs_with_metadata->back()); - } + new_batch = result.value(); } else { - if (outer_blobs) - outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); + is_finished = true; + return {}; + } - std::lock_guard lock(next_mutex); - blobs_with_metadata = std::move(new_batch); - for (const auto & [_, info] : *blobs_with_metadata) - total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); + for (auto it = new_batch.begin(); it != new_batch.end();) + { + if (!recursive && !re2::RE2::FullMatch(it->relative_path, *matcher)) + it = new_batch.erase(it); + else + ++it; } } - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - std::lock_guard lock(next_mutex); - return (*blobs_with_metadata)[current_index]; + index.store(0, std::memory_order_relaxed); + if (!is_initialized) + { + createFilterAST(new_batch.front().relative_path); + is_initialized = true; + } + + if (filter_ast) + { + auto block = virtual_header.cloneEmpty(); + for (size_t i = 0; i < new_batch.size(); ++i) + addPathToVirtualColumns(block, fs::path(container) / new_batch[i].relative_path, i); + + VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); + const auto & idxs = typeid_cast(*block.getByName("_idx").column); + + blobs_with_metadata.clear(); + for (UInt64 idx : idxs.getData()) + { + total_size.fetch_add(new_batch[idx].metadata.size_bytes, std::memory_order_relaxed); + blobs_with_metadata.emplace_back(std::move(new_batch[idx])); + if (outer_blobs) + outer_blobs->emplace_back(blobs_with_metadata.back()); + } + } + else + { + if (outer_blobs) + outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); + + blobs_with_metadata = std::move(new_batch); + for (const auto & [_, info] : blobs_with_metadata) + total_size.fetch_add(info.size_bytes, std::memory_order_relaxed); + } } + + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= blobs_with_metadata.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Index out of bound for blob metadata"); + return blobs_with_metadata[current_index]; } -size_t StorageAzureBlobSource::Iterator::getTotalSize() const +size_t StorageAzureBlobSource::GlobIterator::getTotalSize() const { return total_size.load(std::memory_order_relaxed); } -void StorageAzureBlobSource::Iterator::createFilterAST(const String & any_key) +void StorageAzureBlobSource::GlobIterator::createFilterAST(const String & any_key) { if (!query || !virtual_header) return; @@ -995,6 +923,78 @@ void StorageAzureBlobSource::Iterator::createFilterAST(const String & any_key) } +StorageAzureBlobSource::KeysIterator::KeysIterator( + AzureObjectStorage * object_storage_, + const std::string & container_, + Strings keys_, + ASTPtr query_, + const Block & virtual_header_, + ContextPtr context_, + RelativePathsWithMetadata * outer_blobs_) + : IIterator(context_) + , object_storage(object_storage_) + , container(container_) + , query(query_) + , virtual_header(virtual_header_) + , outer_blobs(outer_blobs_) +{ + Strings all_keys = keys_; + + /// Create a virtual block with one row to construct filter + if (query && virtual_header && !all_keys.empty()) + { + /// Append "idx" column as the filter result + virtual_header.insert({ColumnUInt64::create(), std::make_shared(), "_idx"}); + + auto block = virtual_header.cloneEmpty(); + addPathToVirtualColumns(block, fs::path(container) / all_keys.front(), 0); + + VirtualColumnUtils::prepareFilterBlockWithQuery(query, getContext(), block, filter_ast); + + if (filter_ast) + { + block = virtual_header.cloneEmpty(); + for (size_t i = 0; i < all_keys.size(); ++i) + addPathToVirtualColumns(block, fs::path(container) / all_keys[i], i); + + VirtualColumnUtils::filterBlockWithQuery(query, block, getContext(), filter_ast); + const auto & idxs = typeid_cast(*block.getByName("_idx").column); + + Strings filtered_keys; + filtered_keys.reserve(block.rows()); + for (UInt64 idx : idxs.getData()) + filtered_keys.emplace_back(std::move(all_keys[idx])); + + all_keys = std::move(filtered_keys); + } + } + + for (auto && key : all_keys) + { + ObjectMetadata object_metadata = object_storage->getObjectMetadata(key); + total_size += object_metadata.size_bytes; + keys.emplace_back(RelativePathWithMetadata{key, object_metadata}); + } + + if (outer_blobs) + *outer_blobs = keys; +} + +RelativePathWithMetadata StorageAzureBlobSource::KeysIterator::next() +{ + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= keys.size()) + return {}; + + return keys[current_index]; +} + +size_t StorageAzureBlobSource::KeysIterator::getTotalSize() const +{ + return total_size.load(std::memory_order_relaxed); +} + + Chunk StorageAzureBlobSource::generate() { while (true) @@ -1072,7 +1072,7 @@ StorageAzureBlobSource::StorageAzureBlobSource( String compression_hint_, AzureObjectStorage * object_storage_, const String & container_, - std::shared_ptr file_iterator_) + std::shared_ptr file_iterator_) :ISource(getHeader(sample_block_, requested_virtual_columns_)) , WithContext(context_) , requested_virtual_columns(requested_virtual_columns_) @@ -1167,18 +1167,16 @@ ColumnsDescription StorageAzureBlob::getTableStructureFromData( ContextPtr ctx) { RelativePathsWithMetadata read_keys; - std::shared_ptr file_iterator; + std::shared_ptr file_iterator; if (configuration.withGlobs()) { - file_iterator = std::make_shared( - object_storage, configuration.container, std::nullopt, - configuration.blob_path, nullptr, Block{}, ctx, &read_keys); + file_iterator = std::make_shared( + object_storage, configuration.container, configuration.blob_path, nullptr, Block{}, ctx, &read_keys); } else { - file_iterator = std::make_shared( - object_storage, configuration.container, configuration.blobs_paths, - std::nullopt, nullptr, Block{}, ctx, &read_keys); + file_iterator = std::make_shared( + object_storage, configuration.container, configuration.blobs_paths, nullptr, Block{}, ctx, &read_keys); } std::optional columns_from_cache; diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index e2001fa24ae..31b2beb05aa 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -142,28 +142,37 @@ private: class StorageAzureBlobSource : public ISource, WithContext { public: - class Iterator : WithContext + class IIterator : public WithContext { public: - Iterator( + IIterator(ContextPtr context_):WithContext(context_) {} + virtual ~IIterator() = default; + virtual RelativePathWithMetadata next() = 0; + virtual size_t getTotalSize() const = 0; + + RelativePathWithMetadata operator ()() { return next(); } + }; + + class GlobIterator : public IIterator + { + public: + GlobIterator( AzureObjectStorage * object_storage_, const std::string & container_, - std::optional keys_, - std::optional blob_path_with_globs_, + String blob_path_with_globs_, ASTPtr query_, const Block & virtual_header_, ContextPtr context_, RelativePathsWithMetadata * outer_blobs_); - RelativePathWithMetadata next(); - size_t getTotalSize() const; - ~Iterator() = default; + RelativePathWithMetadata next() override; + size_t getTotalSize() const override; + ~GlobIterator() override = default; private: AzureObjectStorage * object_storage; std::string container; - std::optional keys; - std::optional blob_path_with_globs; + String blob_path_with_globs; ASTPtr query; ASTPtr filter_ast; Block virtual_header; @@ -171,7 +180,7 @@ public: std::atomic index = 0; std::atomic total_size = 0; - std::optional blobs_with_metadata; + RelativePathsWithMetadata blobs_with_metadata; RelativePathsWithMetadata * outer_blobs; ObjectStorageIteratorPtr object_storage_iterator; bool recursive{false}; @@ -184,6 +193,37 @@ public: std::mutex next_mutex; }; + class KeysIterator : public IIterator + { + public: + KeysIterator( + AzureObjectStorage * object_storage_, + const std::string & container_, + Strings keys_, + ASTPtr query_, + const Block & virtual_header_, + ContextPtr context_, + RelativePathsWithMetadata * outer_blobs_); + + RelativePathWithMetadata next() override; + size_t getTotalSize() const override; + ~KeysIterator() override = default; + + private: + AzureObjectStorage * object_storage; + std::string container; + RelativePathsWithMetadata keys; + + ASTPtr query; + ASTPtr filter_ast; + Block virtual_header; + + std::atomic index = 0; + std::atomic total_size = 0; + + RelativePathsWithMetadata * outer_blobs; + }; + StorageAzureBlobSource( const std::vector & requested_virtual_columns_, const String & format_, @@ -196,7 +236,7 @@ public: String compression_hint_, AzureObjectStorage * object_storage_, const String & container_, - std::shared_ptr file_iterator_); + std::shared_ptr file_iterator_); ~StorageAzureBlobSource() override; @@ -217,7 +257,7 @@ private: String compression_hint; AzureObjectStorage * object_storage; String container; - std::shared_ptr file_iterator; + std::shared_ptr file_iterator; struct ReaderHolder { diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 0de325ccd14..5f812cbe4fc 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -595,3 +595,19 @@ def test_partition_by_tf(cluster): assert "1,2,3\n" == get_azure_file_content("test_partition_tf_3.csv") assert "3,2,1\n" == get_azure_file_content("test_partition_tf_1.csv") assert "78,43,45\n" == get_azure_file_content("test_partition_tf_45.csv") + +def test_filter_using_file(cluster): + node = cluster.instances["node"] + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + partition_by = "column3" + values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" + filename = "test_partition_tf_{_partition_id}.csv" + + azure_query( + node, + f"INSERT INTO TABLE FUNCTION azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', '{filename}', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') PARTITION BY {partition_by} VALUES {values}", + ) + + query = f"select count(*) from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_partition_tf_*.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') WHERE _file='test_partition_tf_3.csv'" + assert azure_query(node, query) == "1\n" + From b6517217d8fe5c56bf5086a8965d626674cbfd7a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 23 Jun 2023 10:17:20 +0000 Subject: [PATCH 090/171] Automatic style fix --- tests/integration/test_storage_azure_blob_storage/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 5f812cbe4fc..6089466ff5d 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -596,6 +596,7 @@ def test_partition_by_tf(cluster): assert "3,2,1\n" == get_azure_file_content("test_partition_tf_1.csv") assert "78,43,45\n" == get_azure_file_content("test_partition_tf_45.csv") + def test_filter_using_file(cluster): node = cluster.instances["node"] table_format = "column1 UInt32, column2 UInt32, column3 UInt32" @@ -610,4 +611,3 @@ def test_filter_using_file(cluster): query = f"select count(*) from azureBlobStorage('http://azurite1:10000/devstoreaccount1', 'cont', 'test_partition_tf_*.csv', 'devstoreaccount1', 'Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==', 'CSV', 'auto', '{table_format}') WHERE _file='test_partition_tf_3.csv'" assert azure_query(node, query) == "1\n" - From b5ad349c4f4a877b1cc2386a17a77ff65724394c Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Fri, 23 Jun 2023 14:34:24 +0200 Subject: [PATCH 091/171] Removed unwanted atomic variables to non atomic --- src/Storages/StorageAzureBlob.cpp | 4 ++-- src/Storages/StorageAzureBlob.h | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp index 8a1c81e808e..d582cfe6f4c 100644 --- a/src/Storages/StorageAzureBlob.cpp +++ b/src/Storages/StorageAzureBlob.cpp @@ -860,7 +860,7 @@ RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next() } } - index.store(0, std::memory_order_relaxed); + index = 0; if (!is_initialized) { createFilterAST(new_batch.front().relative_path); @@ -896,7 +896,7 @@ RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next() } } - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + size_t current_index = index++; if (current_index >= blobs_with_metadata.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index out of bound for blob metadata"); return blobs_with_metadata[current_index]; diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h index 31b2beb05aa..746c84841ca 100644 --- a/src/Storages/StorageAzureBlob.h +++ b/src/Storages/StorageAzureBlob.h @@ -177,7 +177,7 @@ public: ASTPtr filter_ast; Block virtual_header; - std::atomic index = 0; + size_t index = 0; std::atomic total_size = 0; RelativePathsWithMetadata blobs_with_metadata; @@ -188,8 +188,8 @@ public: std::unique_ptr matcher; void createFilterAST(const String & any_key); - std::atomic is_finished = false; - std::atomic is_initialized = false; + bool is_finished = false; + bool is_initialized = false; std::mutex next_mutex; }; From 598da6207e355142278ddc061e6a191974697438 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Jun 2023 15:49:37 +0000 Subject: [PATCH 092/171] Use inNull and add a comment. --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 7f6edfc7ec4..9c34a149128 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -554,7 +554,10 @@ void MergeTreeIndexConditionSet::traverseAST(ASTPtr & node) const if (atomFromAST(node)) { if (node->as() || node->as()) - node = makeASTFunction("__bitWrapperFunc", makeASTFunction("toUInt8OrDefault", node)); + /// __bitWrapperFunc* uses default implementation for Nullable types + /// Here we additionally convert Null to 0, + /// otherwise condition 'something OR NULL' will always return Null and filter everything. + node = makeASTFunction("__bitWrapperFunc", makeASTFunction("ifNull", node, std::make_shared(Field(0)))); } else node = std::make_shared(UNKNOWN_FIELD); From fad07bf4b83cd6676fc37032041d772c987f1fe5 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 23 Jun 2023 23:42:31 +0200 Subject: [PATCH 093/171] Fix using locks in ProcessList. --- src/Interpreters/ProcessList.cpp | 26 ++++++++++++++++++-------- src/Interpreters/ProcessList.h | 1 + 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index aca474bf152..c8e33b477a1 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -246,6 +246,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q priorities.insert(static_cast(settings.priority)), std::move(thread_group), query_kind, + settings, watch_start_nanoseconds)); increaseQueryKindAmount(query_kind); @@ -342,6 +343,7 @@ QueryStatus::QueryStatus( QueryPriorities::Handle && priority_handle_, ThreadGroupPtr && thread_group_, IAST::QueryKind query_kind_, + const Settings & query_settings_, UInt64 watch_start_nanoseconds) : WithContext(context_) , query(query_) @@ -353,9 +355,11 @@ QueryStatus::QueryStatus( , query_kind(query_kind_) , num_queries_increment(CurrentMetrics::Query) { - auto settings = getContext()->getSettings(); - limits.max_execution_time = settings.max_execution_time; - overflow_mode = settings.timeout_overflow_mode; + /// We have to pass `query_settings_` to this constructor because we can't use `context_->getSettings().max_execution_time` here: + /// a QueryStatus is created with `ProcessList::mutex` locked (see ProcessList::insert) and calling `context_->getSettings()` + /// would lock the context's lock too, whereas holding two those locks simultaneously is not good. + limits.max_execution_time = query_settings_.max_execution_time; + overflow_mode = query_settings_.timeout_overflow_mode; } QueryStatus::~QueryStatus() @@ -601,12 +605,18 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even ProcessList::Info ProcessList::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const { + /// We have to copy `processes` first because `process->getInfo()` below can access the context to get the query settings, + /// and it's better not to keep the process list's lock while doing that. + std::vector processes_copy; + + { + auto lock = safeLock(); + processes_copy.assign(processes.begin(), processes.end()); + } + Info per_query_infos; - - auto lock = safeLock(); - - per_query_infos.reserve(processes.size()); - for (const auto & process : processes) + per_query_infos.reserve(processes_copy.size()); + for (const auto & process : processes_copy) per_query_infos.emplace_back(process->getInfo(get_thread_list, get_profile_events, get_settings)); return per_query_infos; diff --git a/src/Interpreters/ProcessList.h b/src/Interpreters/ProcessList.h index b593bcef395..e5a61497ff2 100644 --- a/src/Interpreters/ProcessList.h +++ b/src/Interpreters/ProcessList.h @@ -164,6 +164,7 @@ public: QueryPriorities::Handle && priority_handle_, ThreadGroupPtr && thread_group_, IAST::QueryKind query_kind_, + const Settings & query_settings_, UInt64 watch_start_nanoseconds); ~QueryStatus(); From 786987a29a67bb8e26685b1856bf52a20c142ef1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 05:17:43 +0200 Subject: [PATCH 094/171] Remove garbage from function `transform` --- base/base/bit_cast.h | 6 ++ src/Functions/transform.cpp | 161 +++++++++++++++++------------------- 2 files changed, 81 insertions(+), 86 deletions(-) diff --git a/base/base/bit_cast.h b/base/base/bit_cast.h index 5373ead36e8..4783a84586b 100644 --- a/base/base/bit_cast.h +++ b/base/base/bit_cast.h @@ -7,7 +7,13 @@ /** Returns value `from` converted to type `To` while retaining bit representation. * `To` and `From` must satisfy `CopyConstructible`. + * * In contrast to std::bit_cast can cast types of different width. + * + * Note: for signed types of narrower size, the casted result is zero-extended + * instead of sign-extended as with regular static_cast. + * For example, -1 Int8 (represented as 0xFF) bit_casted to UInt64 + * gives 255 (represented as 0x00000000000000FF) instead of 0xFFFFFFFFFFFFFFFF */ template std::decay_t bit_cast(const From & from) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 9970012ef4a..d8b6416e429 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -16,6 +17,8 @@ #include #include #include +#include + namespace DB { @@ -29,28 +32,26 @@ namespace ErrorCodes namespace { - /** transform(x, from_array, to_array[, default]) - convert x according to an explicitly passed match. - */ /** transform(x, [from...], [to...], default) - * - converts the values according to the explicitly specified mapping. - * - * x - what to transform. - * from - a constant array of values for the transformation. - * to - a constant array of values into which values from `from` must be transformed. - * default - what value to use if x is not equal to any of the values in `from`. - * `from` and `to` - arrays of the same size. - * - * Types: - * transform(T, Array(T), Array(U), U) -> U - * - * transform(x, [from...], [to...]) - * - if `default` is not specified, then for values of `x` for which there is no corresponding element in `from`, the unchanged value of `x` is returned. - * - * Types: - * transform(T, Array(T), Array(T)) -> T - * - * Note: the implementation is rather cumbersome. - */ + * - converts the values according to the explicitly specified mapping. + * + * x - what to transform. + * from - a constant array of values for the transformation. + * to - a constant array of values into which values from `from` must be transformed. + * default - what value to use if x is not equal to any of the values in `from`. + * `from` and `to` - arrays of the same size. + * + * Types: + * transform(T, Array(T), Array(U), U) -> U + * + * transform(x, [from...], [to...]) + * - if `default` is not specified, then for values of `x` for which there is no corresponding element in `from`, the unchanged value of `x` is returned. + * + * Types: + * transform(T, Array(T), Array(T)) -> T + * + * Note: the implementation is rather cumbersome. + */ class FunctionTransform : public IFunction { public: @@ -236,7 +237,7 @@ namespace { const auto * it = table.find(bit_cast(pod[i])); if (it) - column_result.insertFrom(*cache.to_columns, it->getMapped()); + column_result.insertFrom(*cache.to_column, it->getMapped()); else if (cache.default_column) column_result.insertFrom(*cache.default_column, 0); else if (default_non_const) @@ -259,7 +260,7 @@ namespace out_offs.resize(size); auto & out_chars = out->getChars(); - const auto * to_col = reinterpret_cast(cache.to_columns.get()); + const auto * to_col = reinterpret_cast(cache.to_column.get()); const auto & to_chars = to_col->getChars(); const auto & to_offs = to_col->getOffsets(); const auto & table = *cache.table_num_to_idx; @@ -340,7 +341,7 @@ namespace if constexpr (std::is_same_v, T> || std::is_same_v, T>) out_scale = out->getScale(); - const auto & to_pod = reinterpret_cast(cache.to_columns.get())->getData(); + const auto & to_pod = reinterpret_cast(cache.to_column.get())->getData(); const auto & table = *cache.table_num_to_idx; if (cache.default_column) { @@ -427,7 +428,7 @@ namespace current_offset = offsets[i]; const auto * it = table.find(ref); if (it) - column_result.insertFrom(*cache.to_columns, it->getMapped()); + column_result.insertFrom(*cache.to_column, it->getMapped()); else if (cache.default_column) column_result.insertFrom(*cache.default_column, 0); else if (default_non_const) @@ -453,7 +454,7 @@ namespace out_offs.resize(size); auto & out_chars = out->getChars(); - const auto * to_col = reinterpret_cast(cache.to_columns.get()); + const auto * to_col = reinterpret_cast(cache.to_column.get()); const auto & to_chars = to_col->getChars(); const auto & to_offs = to_col->getOffsets(); @@ -542,7 +543,7 @@ namespace const size_t size = offsets.size(); out_pod.resize(size); - const auto & to_pod = reinterpret_cast(cache.to_columns.get())->getData(); + const auto & to_pod = reinterpret_cast(cache.to_column.get())->getData(); const auto & table = *cache.table_string_to_idx; if (cache.default_column) { @@ -597,11 +598,10 @@ namespace std::unique_ptr table_num_to_idx; std::unique_ptr table_string_to_idx; - ColumnPtr to_columns; + ColumnPtr from_column; + ColumnPtr to_column; ColumnPtr default_column; - Arena string_pool; - std::atomic initialized{false}; std::mutex mutex; }; @@ -609,27 +609,6 @@ namespace mutable Cache cache; - static UInt64 bitCastToUInt64(const Field & x) - { - switch (x.getType()) - { - case Field::Types::UInt64: - return x.get(); - case Field::Types::Int64: - return x.get(); - case Field::Types::Float64: - return std::bit_cast(x.get()); - case Field::Types::Bool: - return x.get(); - case Field::Types::Decimal32: - return x.get>().getValue(); - case Field::Types::Decimal64: - return x.get>().getValue(); - default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type in function 'transform'"); - } - } - static void checkAllowedType(const DataTypePtr & type) { if (type->isNullable()) @@ -656,33 +635,49 @@ namespace /// Can be called from different threads. It works only on the first call. void initialize(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const { - const ColumnConst * array_from = checkAndGetColumnConst(arguments[1].column.get()); - const ColumnConst * array_to = checkAndGetColumnConst(arguments[2].column.get()); + if (cache.initialized) + return; + + const DataTypePtr & from_type = arguments[0].type; + + const ColumnArray * array_from = checkAndGetColumnConstData(arguments[1].column.get()); + const ColumnArray * array_to = checkAndGetColumnConstData(arguments[2].column.get()); if (!array_from || !array_to) throw Exception( ErrorCodes::ILLEGAL_COLUMN, "Second and third arguments of function {} must be constant arrays.", getName()); - if (cache.initialized) - return; + std::lock_guard lock(cache.mutex); - const auto & from = array_from->getValue(); - const size_t size = from.size(); + ColumnPtr from_column_or_null_ptr = castColumnAccurateOrNull( + { + array_from->getDataPtr(), + typeid_cast(*arguments[1].type).getNestedType(), + arguments[1].name + }, + from_type); + + const ColumnNullable & from_column_or_null = assert_cast(*from_column_or_null_ptr); + + cache.from_column = from_column_or_null.getNestedColumnPtr(); + + cache.to_column = castColumn( + { + array_to->getDataPtr(), + typeid_cast(*arguments[2].type).getNestedType(), + arguments[2].name + }, + result_type); + + const size_t size = cache.from_column->size(); if (0 == size) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty arrays are illegal in function {}", getName()); - std::lock_guard lock(cache.mutex); - - if (cache.initialized) - return; - - const auto & to = array_to->getValue(); - if (size != to.size()) + if (cache.to_column->size() != size) throw Exception( ErrorCodes::BAD_ARGUMENTS, "Second and third arguments of function {} must be arrays of same size", getName()); /// Whether the default value is set. - if (arguments.size() == 4) { const IColumn * default_col = arguments[3].column.get(); @@ -702,20 +697,20 @@ namespace /// Note: Doesn't check the duplicates in the `from` array. - const IDataType & from_type = *arguments[0].type; - - if (from[0].getType() != Field::Types::String) + if (isNativeInteger(from_type)) { cache.table_num_to_idx = std::make_unique(); auto & table = *cache.table_num_to_idx; for (size_t i = 0; i < size; ++i) { - Field key = convertFieldToType(from[i], from_type); - if (key.isNull()) - continue; - - /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 - table[bitCastToUInt64(key)] = i; + if (!from_column_or_null.isNullAt(i)) + { + /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 + StringRef ref = cache.from_column->getDataAt(i); + UInt64 key = 0; + memcpy(&key, ref.data, ref.size); + table[key] = i; + } } } else @@ -724,23 +719,17 @@ namespace auto & table = *cache.table_string_to_idx; for (size_t i = 0; i < size; ++i) { - const String & str_from = from[i].get(); - StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1}; - table[ref] = i; + if (!from_column_or_null.isNullAt(i)) + { + StringRef ref = cache.from_column->getDataAt(i); + table[ref] = i; + } } } - auto to_columns = result_type->createColumn(); - for (size_t i = 0; i < size; ++i) - { - Field to_value = convertFieldToType(to[i], *result_type); - to_columns->insert(to_value); - } - cache.to_columns = std::move(to_columns); - cache.initialized = true; } -}; + }; } From 396eb704269b7d67cfbf654d8b111d826292ddc2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 05:48:08 +0200 Subject: [PATCH 095/171] Remove garbage from function `transform` --- src/Functions/transform.cpp | 39 ++++++++++--------- .../02797_transform_narrow_types.reference | 6 +++ .../02797_transform_narrow_types.sql | 6 +++ 3 files changed, 32 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/02797_transform_narrow_types.reference create mode 100644 tests/queries/0_stateless/02797_transform_narrow_types.sql diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index d8b6416e429..3e4ca412491 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -14,10 +14,8 @@ #include #include #include -#include #include #include -#include namespace DB @@ -28,6 +26,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; + extern const int LOGICAL_ERROR; } namespace @@ -260,14 +259,14 @@ namespace out_offs.resize(size); auto & out_chars = out->getChars(); - const auto * to_col = reinterpret_cast(cache.to_column.get()); + const auto * to_col = assert_cast(cache.to_column.get()); const auto & to_chars = to_col->getChars(); const auto & to_offs = to_col->getOffsets(); const auto & table = *cache.table_num_to_idx; if (cache.default_column) { - const auto * def = reinterpret_cast(cache.default_column.get()); + const auto * def = assert_cast(cache.default_column.get()); const auto & def_chars = def->getChars(); const auto & def_offs = def->getOffsets(); const auto * def_data = def_chars.data(); @@ -276,7 +275,7 @@ namespace } else { - const auto * def = reinterpret_cast(default_non_const.get()); + const auto * def = assert_cast(default_non_const.get()); const auto & def_chars = def->getChars(); const auto & def_offs = def->getOffsets(); executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, size); @@ -341,16 +340,16 @@ namespace if constexpr (std::is_same_v, T> || std::is_same_v, T>) out_scale = out->getScale(); - const auto & to_pod = reinterpret_cast(cache.to_column.get())->getData(); + const auto & to_pod = assert_cast(cache.to_column.get())->getData(); const auto & table = *cache.table_num_to_idx; if (cache.default_column) { - const auto const_def = reinterpret_cast(cache.default_column.get())->getData()[0]; + const auto const_def = assert_cast(cache.default_column.get())->getData()[0]; executeNumToNumHelper(table, pod, out_pod, to_pod, const_def, size, out_scale, out_scale); } else if (default_non_const) { - const auto & nconst_def = reinterpret_cast(default_non_const.get())->getData(); + const auto & nconst_def = assert_cast(default_non_const.get())->getData(); executeNumToNumHelper(table, pod, out_pod, to_pod, nconst_def, size, out_scale, out_scale); } else @@ -424,7 +423,7 @@ namespace ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - const StringRef ref{&data[current_offset], offsets[i] - current_offset}; + const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1}; current_offset = offsets[i]; const auto * it = table.find(ref); if (it) @@ -454,14 +453,14 @@ namespace out_offs.resize(size); auto & out_chars = out->getChars(); - const auto * to_col = reinterpret_cast(cache.to_column.get()); + const auto * to_col = assert_cast(cache.to_column.get()); const auto & to_chars = to_col->getChars(); const auto & to_offs = to_col->getOffsets(); const auto & table = *cache.table_string_to_idx; if (cache.default_column) { - const auto * def = reinterpret_cast(cache.default_column.get()); + const auto * def = assert_cast(cache.default_column.get()); const auto & def_chars = def->getChars(); const auto & def_offs = def->getOffsets(); const auto * def_data = def_chars.data(); @@ -470,7 +469,7 @@ namespace } else if (default_non_const) { - const auto * def = reinterpret_cast(default_non_const.get()); + const auto * def = assert_cast(default_non_const.get()); const auto & def_chars = def->getChars(); const auto & def_offs = def->getOffsets(); executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, size); @@ -501,7 +500,7 @@ namespace { const char8_t * to = nullptr; size_t to_size = 0; - const StringRef ref{&data[current_offset], offsets[i] - current_offset}; + const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1}; current_offset = offsets[i]; const auto * it = table.find(ref); if (it) @@ -543,16 +542,16 @@ namespace const size_t size = offsets.size(); out_pod.resize(size); - const auto & to_pod = reinterpret_cast(cache.to_column.get())->getData(); + const auto & to_pod = assert_cast(cache.to_column.get())->getData(); const auto & table = *cache.table_string_to_idx; if (cache.default_column) { - const auto const_def = reinterpret_cast(cache.default_column.get())->getData()[0]; + const auto const_def = assert_cast(cache.default_column.get())->getData()[0]; executeStringToNumHelper(table, data, offsets, out_pod, to_pod, const_def, size); } else { - const auto & nconst_def = reinterpret_cast(default_non_const.get())->getData(); + const auto & nconst_def = assert_cast(default_non_const.get())->getData(); executeStringToNumHelper(table, data, offsets, out_pod, to_pod, nconst_def, size); } return true; @@ -571,7 +570,7 @@ namespace ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - const StringRef ref{&data[current_offset], offsets[i] - current_offset}; + const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1}; current_offset = offsets[i]; const auto * it = table.find(ref); if (it) @@ -697,7 +696,7 @@ namespace /// Note: Doesn't check the duplicates in the `from` array. - if (isNativeInteger(from_type)) + if (from_type->isValueRepresentedByNumber()) { cache.table_num_to_idx = std::make_unique(); auto & table = *cache.table_num_to_idx; @@ -713,7 +712,7 @@ namespace } } } - else + else if (from_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) { cache.table_string_to_idx = std::make_unique(); auto & table = *cache.table_string_to_idx; @@ -726,6 +725,8 @@ namespace } } } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected data type {} as the first argument in function `transform`", from_type->getName()); cache.initialized = true; } diff --git a/tests/queries/0_stateless/02797_transform_narrow_types.reference b/tests/queries/0_stateless/02797_transform_narrow_types.reference new file mode 100644 index 00000000000..5a2bd60ca83 --- /dev/null +++ b/tests/queries/0_stateless/02797_transform_narrow_types.reference @@ -0,0 +1,6 @@ +f +s +11 +11 +33 +3 diff --git a/tests/queries/0_stateless/02797_transform_narrow_types.sql b/tests/queries/0_stateless/02797_transform_narrow_types.sql new file mode 100644 index 00000000000..56a02fb8af7 --- /dev/null +++ b/tests/queries/0_stateless/02797_transform_narrow_types.sql @@ -0,0 +1,6 @@ +SELECT transform(-1, [-1, 2], ['f', 's'], 'g'); +SELECT transform(2, [-1, 2], ['f', 's'], 'g'); +SELECT transform(-1, [-1, 2], [11, 22], 33); +SELECT transform(-1, [-1, 2], [11, 22]); +SELECT transform(3, [-1, 2], [11, 22], 33); +SELECT transform(3, [-1, 2], [11, 22]); From cb2d3954103fbc2d1230a3122488602de56a4313 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 06:52:28 +0200 Subject: [PATCH 096/171] Add generic implementation of function `transform` --- src/Functions/transform.cpp | 137 +++++++++++++----- .../02798_generic_transform.reference | 38 +++++ .../0_stateless/02798_generic_transform.sql | 12 ++ 3 files changed, 148 insertions(+), 39 deletions(-) create mode 100644 tests/queries/0_stateless/02798_generic_transform.reference create mode 100644 tests/queries/0_stateless/02798_generic_transform.sql diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 3e4ca412491..e2eab1d02fc 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -79,15 +80,6 @@ namespace args_size); const DataTypePtr & type_x = arguments[0]; - const auto & type_x_nn = removeNullable(type_x); - - if (!type_x_nn->isValueRepresentedByNumber() && !isString(type_x_nn) && !isNothing(type_x_nn)) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Unsupported type {} of first argument " - "of function {}, must be numeric type or Date/DateTime or String", - type_x->getName(), - getName()); const DataTypeArray * type_arr_from = checkAndGetDataType(arguments[1].get()); @@ -99,14 +91,13 @@ namespace const auto type_arr_from_nested = type_arr_from->getNestedType(); - if ((type_x->isValueRepresentedByNumber() != type_arr_from_nested->isValueRepresentedByNumber()) - || (isString(type_x) != isString(type_arr_from_nested))) + auto src = tryGetLeastSupertype(DataTypes{type_x, type_arr_from_nested}); + if (!src) { throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument and elements of array " - "of second argument of function {} must have compatible types: " - "both numeric or both strings.", + "of the second argument of function {} must have compatible types", getName()); } @@ -157,8 +148,8 @@ namespace } } - ColumnPtr - executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + ColumnPtr executeImpl( + const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { initialize(arguments, result_type); @@ -172,22 +163,36 @@ namespace default_non_const = castColumn(arguments[3], result_type); auto column_result = result_type->createColumn(); - if (!executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeNum>(in, *column_result, default_non_const) - && !executeString(in, *column_result, default_non_const)) + if (cache.table_num_to_idx) { - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName()); + if (!executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const) + && !executeNum>(in, *column_result, default_non_const)) + { + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName()); + } } + else if (cache.table_string_to_idx) + { + if (!executeString(in, *column_result, default_non_const)) + executeContiguous(in, *column_result, default_non_const); + } + else if (cache.table_anything_to_idx) + { + executeAnything(in, *column_result, default_non_const); + } + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized"); + return column_result; } @@ -204,6 +209,47 @@ namespace return impl->execute(args, result_type, input_rows_count); } + void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const + { + const size_t size = in->size(); + const auto & table = *cache.table_anything_to_idx; + column_result.reserve(size); + for (size_t i = 0; i < size; ++i) + { + SipHash hash; + in->updateHashWithValue(i, hash); + + const auto * it = table.find(hash.get128()); + if (it) + column_result.insertFrom(*cache.to_column, it->getMapped()); + else if (cache.default_column) + column_result.insertFrom(*cache.default_column, 0); + else if (default_non_const) + column_result.insertFrom(*default_non_const, i); + else + column_result.insertFrom(*in, i); + } + } + + void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const + { + const size_t size = in->size(); + const auto & table = *cache.table_string_to_idx; + column_result.reserve(size); + for (size_t i = 0; i < size; ++i) + { + const auto * it = table.find(in->getDataAt(i)); + if (it) + column_result.insertFrom(*cache.to_column, it->getMapped()); + else if (cache.default_column) + column_result.insertFrom(*cache.default_column, 0); + else if (default_non_const) + column_result.insertFrom(*default_non_const, i); + else + column_result.insertFrom(*in, i); + } + } + template bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const) const { @@ -593,9 +639,11 @@ namespace { using NumToIdx = HashMap>; using StringToIdx = HashMap; + using AnythingToIdx = HashMap; std::unique_ptr table_num_to_idx; std::unique_ptr table_string_to_idx; + std::unique_ptr table_anything_to_idx; ColumnPtr from_column; ColumnPtr to_column; @@ -648,18 +696,16 @@ namespace std::lock_guard lock(cache.mutex); - ColumnPtr from_column_or_null_ptr = castColumnAccurateOrNull( + const ColumnPtr & from_column_uncasted = array_from->getDataPtr(); + + cache.from_column = castColumn( { - array_from->getDataPtr(), + from_column_uncasted, typeid_cast(*arguments[1].type).getNestedType(), arguments[1].name }, from_type); - const ColumnNullable & from_column_or_null = assert_cast(*from_column_or_null_ptr); - - cache.from_column = from_column_or_null.getNestedColumnPtr(); - cache.to_column = castColumn( { array_to->getDataPtr(), @@ -696,13 +742,14 @@ namespace /// Note: Doesn't check the duplicates in the `from` array. - if (from_type->isValueRepresentedByNumber()) + WhichDataType which(from_type); + if (isNativeNumber(which) || which.isDecimal32() || which.isDecimal64()) { cache.table_num_to_idx = std::make_unique(); auto & table = *cache.table_num_to_idx; for (size_t i = 0; i < size; ++i) { - if (!from_column_or_null.isNullAt(i)) + if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i])) { /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 StringRef ref = cache.from_column->getDataAt(i); @@ -718,7 +765,7 @@ namespace auto & table = *cache.table_string_to_idx; for (size_t i = 0; i < size; ++i) { - if (!from_column_or_null.isNullAt(i)) + if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i])) { StringRef ref = cache.from_column->getDataAt(i); table[ref] = i; @@ -726,7 +773,19 @@ namespace } } else - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected data type {} as the first argument in function `transform`", from_type->getName()); + { + cache.table_anything_to_idx = std::make_unique(); + auto & table = *cache.table_anything_to_idx; + for (size_t i = 0; i < size; ++i) + { + if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i])) + { + SipHash hash; + cache.from_column->updateHashWithValue(i, hash); + table[hash.get128()] = i; + } + } + } cache.initialized = true; } diff --git a/tests/queries/0_stateless/02798_generic_transform.reference b/tests/queries/0_stateless/02798_generic_transform.reference new file mode 100644 index 00000000000..2140c3218b3 --- /dev/null +++ b/tests/queries/0_stateless/02798_generic_transform.reference @@ -0,0 +1,38 @@ +def +def +def +hello +def +world +def +abc! +def +def + + + +hello + +world + +abc + + + + + +hello + +world + +abc + + +123 +2023-03-03 00:00:00.000 +2023-02-02 00:00:00.000 +2023-01-01 00:00:00.000 +1 1 +42 42 +42 +42 diff --git a/tests/queries/0_stateless/02798_generic_transform.sql b/tests/queries/0_stateless/02798_generic_transform.sql new file mode 100644 index 00000000000..6317d83fb29 --- /dev/null +++ b/tests/queries/0_stateless/02798_generic_transform.sql @@ -0,0 +1,12 @@ +SELECT transform((number, toString(number)), [(3, '3'), (5, '5'), (7, '7')], ['hello', 'world', 'abc!'], 'def') FROM system.numbers LIMIT 10; +SELECT transform(toNullable(toInt256(number)), [3, 5, 7], ['hello', 'world', 'abc'], '') FROM system.numbers LIMIT 10; +SELECT transform(toUInt256(number), [3, 5, 7], ['hello', 'world', 'abc'], '') FROM system.numbers LIMIT 10; + +select case 1::Nullable(Int32) when 1 then 123 else 0 end; + +SELECT transform(arrayJoin(['c', 'b', 'a']), ['a', 'b'], [toDateTime64('2023-01-01', 3), toDateTime64('2023-02-02', 3)], toDateTime64('2023-03-03', 3)); + +SELECT transform(1, [1], [toDecimal32(1, 2)]), toDecimal32(1, 2); +select transform(1, [1], [toDecimal32(42, 2)]), toDecimal32(42, 2); +SELECT transform(1, [1], [toDecimal32(42, 2)], 0); +SELECT transform(1, [1], [toDecimal32(42, 2)], toDecimal32(0, 2)); From 3746e274b8fab462621adaf9d1d0b4902a3c7252 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 07:05:31 +0200 Subject: [PATCH 097/171] Support empty arrays --- src/Functions/transform.cpp | 15 ++++++++-- .../02799_transform_empty_arrays.reference | 30 +++++++++++++++++++ .../02799_transform_empty_arrays.sql | 3 ++ 3 files changed, 46 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02799_transform_empty_arrays.reference create mode 100644 tests/queries/0_stateless/02799_transform_empty_arrays.sql diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index e2eab1d02fc..a4667b1cea0 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -163,7 +163,13 @@ namespace default_non_const = castColumn(arguments[3], result_type); auto column_result = result_type->createColumn(); - if (cache.table_num_to_idx) + if (cache.is_empty) + { + return default_non_const + ? default_non_const + : castColumn(arguments[0], result_type); + } + else if (cache.table_num_to_idx) { if (!executeNum>(in, *column_result, default_non_const) && !executeNum>(in, *column_result, default_non_const) @@ -645,6 +651,8 @@ namespace std::unique_ptr table_string_to_idx; std::unique_ptr table_anything_to_idx; + bool is_empty = false; + ColumnPtr from_column; ColumnPtr to_column; ColumnPtr default_column; @@ -716,7 +724,10 @@ namespace const size_t size = cache.from_column->size(); if (0 == size) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty arrays are illegal in function {}", getName()); + { + cache.is_empty = true; + return; + } if (cache.to_column->size() != size) throw Exception( diff --git a/tests/queries/0_stateless/02799_transform_empty_arrays.reference b/tests/queries/0_stateless/02799_transform_empty_arrays.reference new file mode 100644 index 00000000000..fd44083947e --- /dev/null +++ b/tests/queries/0_stateless/02799_transform_empty_arrays.reference @@ -0,0 +1,30 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +Hello +Hello +Hello +Hello +Hello +Hello +Hello +Hello +Hello +Hello +Hello 0 +Hello 1 +Hello 2 +Hello 3 +Hello 4 +Hello 5 +Hello 6 +Hello 7 +Hello 8 +Hello 9 diff --git a/tests/queries/0_stateless/02799_transform_empty_arrays.sql b/tests/queries/0_stateless/02799_transform_empty_arrays.sql new file mode 100644 index 00000000000..84e3e9d2930 --- /dev/null +++ b/tests/queries/0_stateless/02799_transform_empty_arrays.sql @@ -0,0 +1,3 @@ +SELECT transform(number, [], [1]) FROM numbers(10); +SELECT transform(number, [], [], 'Hello') FROM numbers(10); +SELECT transform(number, [], [], 'Hello ' || number::String) FROM numbers(10); From f48c1605da9a3e06a6cca2bfa5cf63b6b518659e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 07:08:44 +0200 Subject: [PATCH 098/171] Add a test --- .../02800_transform_alter.reference | 6 +++ .../0_stateless/02800_transform_alter.sql | 43 +++++++++++++++++++ 2 files changed, 49 insertions(+) create mode 100644 tests/queries/0_stateless/02800_transform_alter.reference create mode 100644 tests/queries/0_stateless/02800_transform_alter.sql diff --git a/tests/queries/0_stateless/02800_transform_alter.reference b/tests/queries/0_stateless/02800_transform_alter.reference new file mode 100644 index 00000000000..cae8aeba20f --- /dev/null +++ b/tests/queries/0_stateless/02800_transform_alter.reference @@ -0,0 +1,6 @@ +1 a1 a1 +2 a2 b2 +3 a3 b3 +1 a1 +2 b2 +3 b3 diff --git a/tests/queries/0_stateless/02800_transform_alter.sql b/tests/queries/0_stateless/02800_transform_alter.sql new file mode 100644 index 00000000000..7458f51c5a4 --- /dev/null +++ b/tests/queries/0_stateless/02800_transform_alter.sql @@ -0,0 +1,43 @@ +DROP TABLE IF EXISTS test_xy; +DROP TABLE IF EXISTS updates; + +CREATE TABLE test_xy +( + `x` Int32, + `y` String +) +ENGINE = MergeTree +ORDER BY x; + +CREATE TABLE updates +( + `x` Int32, + `y` String +) +ENGINE = MergeTree +ORDER BY x; + +INSERT INTO test_xy(x, y) VALUES (1, 'a1'), (2, 'a2'), (3, 'a3'); +INSERT INTO updates(x, y) VALUES (2, 'b2'), (3, 'b3'); + +SELECT x, y, + transform(x, + (select groupArray(x) from (select x, y from updates order by x) t1), + (select groupArray(y) from (select x, y from updates order by x) t2), + y) +FROM test_xy +WHERE 1 ORDER BY x, y; + +SET mutations_sync = 1; +ALTER table test_xy + UPDATE + y = transform(x, + (select groupArray(x) from (select x, y from updates order by x) t1), + (select groupArray(y) from (select x, y from updates order by x) t2), + y) + WHERE 1; + +SELECT * FROM test_xy ORDER BY x, y; + +DROP TABLE test_xy; +DROP TABLE updates; From ed3ead3493b936346cda744dbbb0a18f19964918 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 07:17:50 +0200 Subject: [PATCH 099/171] Add support for Nullable --- src/Functions/transform.cpp | 6 ++ .../02801_transform_nullable.reference | 65 +++++++++++++++++++ .../0_stateless/02801_transform_nullable.sql | 12 ++++ 3 files changed, 83 insertions(+) create mode 100644 tests/queries/0_stateless/02801_transform_nullable.reference create mode 100644 tests/queries/0_stateless/02801_transform_nullable.sql diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index a4667b1cea0..db41c6fa081 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -695,6 +695,12 @@ namespace const DataTypePtr & from_type = arguments[0].type; + if (from_type->onlyNull()) + { + cache.is_empty = true; + return; + } + const ColumnArray * array_from = checkAndGetColumnConstData(arguments[1].column.get()); const ColumnArray * array_to = checkAndGetColumnConstData(arguments[2].column.get()); diff --git a/tests/queries/0_stateless/02801_transform_nullable.reference b/tests/queries/0_stateless/02801_transform_nullable.reference new file mode 100644 index 00000000000..b51587c094c --- /dev/null +++ b/tests/queries/0_stateless/02801_transform_nullable.reference @@ -0,0 +1,65 @@ +1970-01-01 00:00:01.000 +1970-01-01 00:00:02.000 +1970-01-01 00:00:00.000 +\N +\N +\N +hello +\N +world +\N +abc +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +\N +-1 +-1 +-1 +111 +-1 +222 +-1 +\N +-1 +-1 +-1.1 +-1.1 +-1.1 +\N +-1.1 +222 +-1.1 +333 +-1.1 +-1.1 +1 +1 +1 +\N +1 +\N +1 +\N +1 +1 +Остальные +\N diff --git a/tests/queries/0_stateless/02801_transform_nullable.sql b/tests/queries/0_stateless/02801_transform_nullable.sql new file mode 100644 index 00000000000..bcf4adba2f8 --- /dev/null +++ b/tests/queries/0_stateless/02801_transform_nullable.sql @@ -0,0 +1,12 @@ +select transform('a', ['a', 'b'], [toDateTime64(1, 3, 'UTC'), toDateTime64(2, 3, 'UTC')], toDateTime64(0, 3, 'UTC')); +select transform(2, [1, 2], [toDateTime64(1, 3, 'UTC'), toDateTime64(2, 3, 'UTC')], toDateTime64(0, 3, 'UTC')); +select transform(null, [1, 2], [toDateTime64(1, 3, 'UTC'), toDateTime64(2, 3, 'UTC')], toDateTime64(0, 3, 'UTC')); + +SELECT transform(number, [3, 5, 7], ['hello', 'world', 'abc'], null) FROM system.numbers LIMIT 10; +SELECT transform(null, ['3', '5', '7'], ['hello', 'world', 'abc'], null) FROM system.numbers LIMIT 10; +SELECT transform(null, [null, null, null], [null, null, null], null) FROM system.numbers LIMIT 10; +SELECT transform(toString(number), ['3', '5', '7'], [111, 222, null], -1) FROM system.numbers LIMIT 10; +SELECT transform(toString(number), ['3', '5', '7'], [null, 222, 333], materialize(-1.1)) FROM system.numbers LIMIT 10; +SELECT transform(toString(number), ['3', '5', '7'], [null, null, null], materialize(1)) FROM system.numbers LIMIT 10; +SELECT transform(1, [2, 3], ['Meta.ua', null], materialize('Остальные')) AS title; +SELECT transform(2, [2, 3], [null, 'Google'], materialize('Остальные')) AS title; From 64a42c7d3a7738a6055de215daf5a113495e45f7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 07:19:45 +0200 Subject: [PATCH 100/171] More tests --- .../0_stateless/02801_transform_nullable.reference | 10 ++++++++++ tests/queries/0_stateless/02801_transform_nullable.sql | 2 ++ 2 files changed, 12 insertions(+) diff --git a/tests/queries/0_stateless/02801_transform_nullable.reference b/tests/queries/0_stateless/02801_transform_nullable.reference index b51587c094c..48d529e7d90 100644 --- a/tests/queries/0_stateless/02801_transform_nullable.reference +++ b/tests/queries/0_stateless/02801_transform_nullable.reference @@ -63,3 +63,13 @@ abc 1 Остальные \N +- +xyz +Hello +- +xyz +World +- +xyz +- +- diff --git a/tests/queries/0_stateless/02801_transform_nullable.sql b/tests/queries/0_stateless/02801_transform_nullable.sql index bcf4adba2f8..95f3c16600a 100644 --- a/tests/queries/0_stateless/02801_transform_nullable.sql +++ b/tests/queries/0_stateless/02801_transform_nullable.sql @@ -10,3 +10,5 @@ SELECT transform(toString(number), ['3', '5', '7'], [null, 222, 333], materializ SELECT transform(toString(number), ['3', '5', '7'], [null, null, null], materialize(1)) FROM system.numbers LIMIT 10; SELECT transform(1, [2, 3], ['Meta.ua', null], materialize('Остальные')) AS title; SELECT transform(2, [2, 3], [null, 'Google'], materialize('Остальные')) AS title; + +SELECT transform(number % 3 = 1 ? NULL : number, [2, 5, NULL], ['Hello', 'World', 'xyz'], '-') FROM numbers(10); From f41183d2098f778381fa1d50ef30411d54c4d73a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 08:22:42 +0200 Subject: [PATCH 101/171] Update test --- .../02497_if_transform_strings_to_enum.reference | 10 ++++++++++ .../0_stateless/02497_if_transform_strings_to_enum.sql | 2 +- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference index a1a653361ee..88f23334d31 100644 --- a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference +++ b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.reference @@ -405,6 +405,16 @@ QUERY id: 0 TABLE id: 7, table_name: system.numbers LIMIT CONSTANT id: 17, constant_value: UInt64_10, constant_value_type: UInt64 +other +other +other +other +other +other +other +other +other +other SELECT transform(number, [NULL], _CAST([\'google\', \'censor.net\', \'yahoo\'], \'Array(Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4))\'), _CAST(\'other\', \'Enum8(\\\'censor.net\\\' = 1, \\\'google\\\' = 2, \\\'other\\\' = 3, \\\'yahoo\\\' = 4)\')) FROM ( diff --git a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.sql b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.sql index 492d42cb6bc..c3db61d1fb2 100644 --- a/tests/queries/0_stateless/02497_if_transform_strings_to_enum.sql +++ b/tests/queries/0_stateless/02497_if_transform_strings_to_enum.sql @@ -33,7 +33,7 @@ SELECT transform(number, [2, 4, 6], ['google', 'censor.net', 'yahoo'], 'other') EXPLAIN SYNTAX SELECT transform(number, [2, 4, 6], ['google', 'censor.net', 'yahoo'], 'other') as value, value FROM system.numbers LIMIT 10; EXPLAIN QUERY TREE run_passes = 1 SELECT transform(number, [2, 4, 6], ['google', 'censor.net', 'yahoo'], 'other') as value, value FROM system.numbers LIMIT 10; -SELECT transform(number, [NULL], ['google', 'censor.net', 'yahoo'], 'other') FROM (SELECT NULL as number FROM system.numbers LIMIT 10); -- { serverError 36 } +SELECT transform(number, [NULL], ['google', 'censor.net', 'yahoo'], 'other') FROM (SELECT NULL as number FROM system.numbers LIMIT 10); EXPLAIN SYNTAX SELECT transform(number, [NULL], ['google', 'censor.net', 'yahoo'], 'other') FROM (SELECT NULL as number FROM system.numbers LIMIT 10); EXPLAIN QUERY TREE run_passes = 1 SELECT transform(number, [NULL], ['google', 'censor.net', 'yahoo'], 'other') FROM (SELECT NULL as number FROM system.numbers LIMIT 10); From 7d55461f8b6aa6b7c59db312e247688d2c31c70f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 08:23:36 +0200 Subject: [PATCH 102/171] Update test --- tests/queries/0_stateless/01733_transform_ubsan.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01733_transform_ubsan.sql b/tests/queries/0_stateless/01733_transform_ubsan.sql index 7c3d8ef653a..f2e89e104d7 100644 --- a/tests/queries/0_stateless/01733_transform_ubsan.sql +++ b/tests/queries/0_stateless/01733_transform_ubsan.sql @@ -1,4 +1,4 @@ SELECT arrayStringConcat(arrayMap(x -> transform(x, [1025, -9223372036854775808, 65537, 257, 1048576, 10, 7, 1048575, 65536], ['censor.net', 'googlegooglegooglegoogle', 'test', '', '', 'hello', 'world', '', 'xyz'], ''), arrayMap(x -> (x % -inf), range(number))), '') FROM system.numbers LIMIT 1025 -FORMAT Null; +FORMAT Null; -- { serverError 43 } From a7febd1c13e7b2d3cd932dcb316660db4cb08a4c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 08:28:23 +0200 Subject: [PATCH 103/171] Backward compatibility --- src/Functions/transform.cpp | 5 ++++- tests/queries/0_stateless/01733_transform_ubsan.sql | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index db41c6fa081..8d6e53c491e 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -92,7 +92,10 @@ namespace const auto type_arr_from_nested = type_arr_from->getNestedType(); auto src = tryGetLeastSupertype(DataTypes{type_x, type_arr_from_nested}); - if (!src) + if (!src + /// Compatibility with previous versions, that allowed even UInt64 with Int64, + /// regardless of ambiguous conversions. + && !isNativeNumber(type_x) && !isNativeNumber(type_arr_from_nested)) { throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, diff --git a/tests/queries/0_stateless/01733_transform_ubsan.sql b/tests/queries/0_stateless/01733_transform_ubsan.sql index f2e89e104d7..7c3d8ef653a 100644 --- a/tests/queries/0_stateless/01733_transform_ubsan.sql +++ b/tests/queries/0_stateless/01733_transform_ubsan.sql @@ -1,4 +1,4 @@ SELECT arrayStringConcat(arrayMap(x -> transform(x, [1025, -9223372036854775808, 65537, 257, 1048576, 10, 7, 1048575, 65536], ['censor.net', 'googlegooglegooglegoogle', 'test', '', '', 'hello', 'world', '', 'xyz'], ''), arrayMap(x -> (x % -inf), range(number))), '') FROM system.numbers LIMIT 1025 -FORMAT Null; -- { serverError 43 } +FORMAT Null; From 3ed86867f8b2c055fab7903619f0a74650742722 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Jun 2023 23:06:13 +0200 Subject: [PATCH 104/171] Remove `sipHash128ReferenceKeyed` --- src/Functions/FunctionsHashing.h | 29 --- src/Functions/FunctionsHashingMisc.cpp | 5 - .../02552_siphash128_reference.sql | 206 ------------------ 3 files changed, 240 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index a4d4fbd085d..db0ff976d63 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -402,34 +402,6 @@ struct SipHash128ReferenceImpl static constexpr bool use_int_hash_for_pods = false; }; -struct SipHash128ReferenceKeyedImpl -{ - static constexpr auto name = "sipHash128ReferenceKeyed"; - using ReturnType = UInt128; - using Key = impl::SipHashKey; - - static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); } - - static UInt128 applyKeyed(const Key & key, const char * begin, size_t size) - { - return sipHash128ReferenceKeyed(key.key0, key.key1, begin, size); - } - - static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2) - { -#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ - UInt128 tmp; - reverseMemcpy(&tmp, &h1, sizeof(UInt128)); - h1 = tmp; - reverseMemcpy(&tmp, &h2, sizeof(UInt128)); - h2 = tmp; -#endif - UInt128 hashes[] = {h1, h2}; - return applyKeyed(key, reinterpret_cast(hashes), 2 * sizeof(UInt128)); - } - - static constexpr bool use_int_hash_for_pods = false; -}; /** Why we need MurmurHash2? * MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash. @@ -1737,7 +1709,6 @@ using FunctionSHA512 = FunctionStringHashFixedString; using FunctionSipHash128 = FunctionAnyHash; using FunctionSipHash128Keyed = FunctionAnyHash; using FunctionSipHash128Reference = FunctionAnyHash; -using FunctionSipHash128ReferenceKeyed = FunctionAnyHash; using FunctionCityHash64 = FunctionAnyHash; using FunctionFarmFingerprint64 = FunctionAnyHash; using FunctionFarmHash64 = FunctionAnyHash; diff --git a/src/Functions/FunctionsHashingMisc.cpp b/src/Functions/FunctionsHashingMisc.cpp index 127c87ba940..56c3c1ed00c 100644 --- a/src/Functions/FunctionsHashingMisc.cpp +++ b/src/Functions/FunctionsHashingMisc.cpp @@ -20,11 +20,6 @@ REGISTER_FUNCTION(Hashing) .examples{{"hash", "SELECT hex(sipHash128Reference('foo', '\\x01', 3))", ""}}, .categories{"Hash"} }); - factory.registerFunction(FunctionDocumentation{ - .description="Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument instead of using a fixed key.", - .examples{{"hash", "SELECT hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912),'foo', '\\x01', 3));", ""}}, - .categories{"Hash"} - }); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/tests/queries/0_stateless/02552_siphash128_reference.sql b/tests/queries/0_stateless/02552_siphash128_reference.sql index c238e51b690..200954c3b57 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.sql +++ b/tests/queries/0_stateless/02552_siphash128_reference.sql @@ -1,207 +1 @@ --- Test Vectors from the SipHash reference C implementation: --- Written by --- Jean-Philippe Aumasson --- Daniel J. Bernstein --- Released under CC0 --- https://github.com/veorq/SipHash/blob/eee7d0d84dc7731df2359b243aa5e75d85f6eaef/vectors.h#L645 - -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - '')); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61))); -select hex(sipHash128ReferenceKeyed((toUInt64(506097522914230528), toUInt64(1084818905618843912)), - char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62))); - --- CH tests -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0)) == sipHash128Reference(char(0)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1)) == sipHash128Reference(char(0, 1)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2)) == sipHash128Reference(char(0, 1, 2)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3)) == sipHash128Reference(char(0, 1, 2, 3)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4)) == sipHash128Reference(char(0, 1, 2, 3, 4)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62)); -select sipHash128ReferenceKeyed((toUInt64(0),toUInt64(0)),char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)) == sipHash128Reference(char(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, 40, 41, 42, 43, 44, 45, 46, 47, 48, 49, 50, 51, 52, 53, 54, 55, 56, 57, 58, 59, 60, 61, 62, 63)); - -select sipHash128ReferenceKeyed((0, 0), '1'); -- { serverError 48 } -select sipHash128ReferenceKeyed(toUInt64(0), '1'); -- { serverError 48 } - SELECT hex(sipHash128Reference()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; -SELECT hex(sipHash128ReferenceKeyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; From 84c592aecdb9cbdba17dc7e3431b14c785da11f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 01:34:15 +0200 Subject: [PATCH 105/171] Update test --- tests/queries/0_stateless/00809_add_days_segfault.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00809_add_days_segfault.sql b/tests/queries/0_stateless/00809_add_days_segfault.sql index 6791439708a..d2d91dd2711 100644 --- a/tests/queries/0_stateless/00809_add_days_segfault.sql +++ b/tests/queries/0_stateless/00809_add_days_segfault.sql @@ -9,4 +9,4 @@ SET send_logs_level = 'fatal'; SELECT ignore(addDays((CAST((96.338) AS DateTime)), -3)); SELECT ignore(subtractDays((CAST((-5263074.47) AS DateTime)), -737895)); SELECT quantileDeterministic([], identity(( SELECT subtractDays((CAST((566450.398706) AS DateTime)), 54) ) )), '\0', []; -- { serverError 43 } -SELECT sequenceCount((CAST((( SELECT NULL ) AS rg, ( SELECT ( SELECT [], ' Date: Sun, 25 Jun 2023 03:22:59 +0200 Subject: [PATCH 106/171] Disable "compile_expressions" setting by default --- src/Core/Settings.h | 2 +- tests/clickhouse-test | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 3d42bd582ed..658f3c8025b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -141,7 +141,7 @@ class IColumn; M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \ M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \ M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \ - M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \ + M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code. This feature has a bug and should not be used.", 0) \ M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \ diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 82aa3639179..4860ce0fac9 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -589,7 +589,7 @@ class SettingsRandomizer: "filesystem_prefetch_step_bytes": lambda: random.choice( [0, "100Mi"] ), # 0 means 'auto' - "compile_expressions": lambda: random.randint(0, 1), + # "compile_expressions": lambda: random.randint(0, 1), - this setting has a bug: https://github.com/ClickHouse/ClickHouse/issues/51264 "compile_aggregate_expressions": lambda: random.randint(0, 1), "compile_sort_description": lambda: random.randint(0, 1), "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), From 171e11f27693b76d4d131c7a00592fd58bfe884a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 04:11:27 +0200 Subject: [PATCH 107/171] Fix "merge_truncate_long" test --- .../01443_merge_truncate_long.reference | 100 ++++++++++++++++++ .../0_stateless/01443_merge_truncate_long.sh | 37 +++++-- 2 files changed, 127 insertions(+), 10 deletions(-) diff --git a/tests/queries/0_stateless/01443_merge_truncate_long.reference b/tests/queries/0_stateless/01443_merge_truncate_long.reference index e69de29bb2d..190423f88f8 100644 --- a/tests/queries/0_stateless/01443_merge_truncate_long.reference +++ b/tests/queries/0_stateless/01443_merge_truncate_long.reference @@ -0,0 +1,100 @@ +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +20 +21 +22 +23 +24 +25 +26 +27 +28 +29 +30 +31 +32 +33 +34 +35 +36 +37 +38 +39 +40 +41 +42 +43 +44 +45 +46 +47 +48 +49 +50 +51 +52 +53 +54 +55 +56 +57 +58 +59 +60 +61 +62 +63 +64 +65 +66 +67 +68 +69 +70 +71 +72 +73 +74 +75 +76 +77 +78 +79 +80 +81 +82 +83 +84 +85 +86 +87 +88 +89 +90 +91 +92 +93 +94 +95 +96 +97 +98 +99 +100 diff --git a/tests/queries/0_stateless/01443_merge_truncate_long.sh b/tests/queries/0_stateless/01443_merge_truncate_long.sh index 00abc48493a..ce867d32c1a 100755 --- a/tests/queries/0_stateless/01443_merge_truncate_long.sh +++ b/tests/queries/0_stateless/01443_merge_truncate_long.sh @@ -3,23 +3,40 @@ set -e +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none + CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=none/g') - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY tuple()" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY ()" -for _ in {1..70}; do - ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" - ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" - ${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE t FINAL" 2>/dev/null & - ${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE t" - ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM t HAVING count() > 0" -done +function thread() +{ + trap 'BREAK=1' 2 + + while [[ -z "${BREAK}" ]] + do + ${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE t FINAL;" 2>&1 | tr -d '\n' | rg -v 'Cancelled merging parts' ||: + done +} + +thread & +pid=$! + +for i in {1..100}; do + echo " + INSERT INTO t VALUES (0); + INSERT INTO t VALUES (0); + TRUNCATE TABLE t; + SELECT count() FROM t HAVING count() > 0; + SELECT ${i}; + " +done | ${CLICKHOUSE_CLIENT} --multiquery + +kill -2 "$pid" wait $CLICKHOUSE_CLIENT -q "DROP TABLE t" From 1482df53cacfee5d102ff7d99b9ba520dcb2267b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 04:14:40 +0200 Subject: [PATCH 108/171] Update reference --- .../02552_siphash128_reference.reference | 129 ------------------ 1 file changed, 129 deletions(-) diff --git a/tests/queries/0_stateless/02552_siphash128_reference.reference b/tests/queries/0_stateless/02552_siphash128_reference.reference index 452e9910660..d00491fd7e5 100644 --- a/tests/queries/0_stateless/02552_siphash128_reference.reference +++ b/tests/queries/0_stateless/02552_siphash128_reference.reference @@ -1,130 +1 @@ -A3817F04BA25A8E66DF67214C7550293 -DA87C1D86B99AF44347659119B22FC45 -8177228DA4A45DC7FCA38BDEF60AFFE4 -9C70B60C5267A94E5F33B6B02985ED51 -F88164C12D9C8FAF7D0F6E7C7BCD5579 -1368875980776F8854527A07690E9627 -14EECA338B208613485EA0308FD7A15E -A1F1EBBED8DBC153C0B84AA61FF08239 -3B62A9BA6258F5610F83E264F31497B4 -264499060AD9BAABC47F8B02BB6D71ED -00110DC378146956C95447D3F3D0FBBA -0151C568386B6677A2B4DC6F81E5DC18 -D626B266905EF35882634DF68532C125 -9869E247E9C08B10D029934FC4B952F7 -31FCEFAC66D7DE9C7EC7485FE4494902 -5493E99933B0A8117E08EC0F97CFC3D9 -6EE2A4CA67B054BBFD3315BF85230577 -473D06E8738DB89854C066C47AE47740 -A426E5E423BF4885294DA481FEAEF723 -78017731CF65FAB074D5208952512EB1 -9E25FC833F2290733E9344A5E83839EB -568E495ABE525A218A2214CD3E071D12 -4A29B54552D16B9A469C10528EFF0AAE -C9D184DDD5A9F5E0CF8CE29A9ABF691C -2DB479AE78BD50D8882A8A178A6132AD -8ECE5F042D5E447B5051B9EACB8D8F6F -9C0B53B4B3C307E87EAEE08678141F66 -ABF248AF69A6EAE4BFD3EB2F129EEB94 -0664DA1668574B88B935F3027358AEF4 -AA4B9DC4BF337DE90CD4FD3C467C6AB7 -EA5C7F471FAF6BDE2B1AD7D4686D2287 -2939B0183223FAFC1723DE4F52C43D35 -7C3956CA5EEAFC3E363E9D556546EB68 -77C6077146F01C32B6B69D5F4EA9FFCF -37A6986CB8847EDF0925F0F1309B54DE -A705F0E69DA9A8F907241A2E923C8CC8 -3DC47D1F29C448461E9E76ED904F6711 -0D62BF01E6FC0E1A0D3C4751C5D3692B -8C03468BCA7C669EE4FD5E084BBEE7B5 -528A5BB93BAF2C9C4473CCE5D0D22BD9 -DF6A301E95C95DAD97AE0CC8C6913BD8 -801189902C857F39E73591285E70B6DB -E617346AC9C231BB3650AE34CCCA0C5B -27D93437EFB721AA401821DCEC5ADF89 -89237D9DED9C5E78D8B1C9B166CC7342 -4A6D8091BF5E7D651189FA94A250B14C -0E33F96055E7AE893FFC0E3DCF492902 -E61C432B720B19D18EC8D84BDC63151B -F7E5AEF549F782CF379055A608269B16 -438D030FD0B7A54FA837F2AD201A6403 -A590D3EE4FBF04E3247E0D27F286423F -5FE2C1A172FE93C4B15CD37CAEF9F538 -2C97325CBD06B36EB2133DD08B3A017C -92C814227A6BCA949FF0659F002AD39E -DCE850110BD8328CFBD50841D6911D87 -67F14984C7DA791248E32BB5922583DA -1938F2CF72D54EE97E94166FA91D2A36 -74481E9646ED49FE0F6224301604698E -57FCA5DE98A9D6D8006438D0583D8A1D -9FECDE1CEFDC1CBED4763674D9575359 -E3040C00EB28F15366CA73CBD872E740 -7697009A6A831DFECCA91C5993670F7A -5853542321F567A005D547A4F04759BD -5150D1772F50834A503E069A973FBD7C -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 1 From 650e80663c11bafdaca32580fc0f3980cd409be8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 05:59:02 +0200 Subject: [PATCH 109/171] Fix MSan report in lowerUTF8/upperUTF8 --- src/Functions/LowerUpperUTF8Impl.h | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index e8d0abae322..f6b18439fd1 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -139,7 +139,7 @@ struct LowerUpperUTF8Impl /// In case partial buffer was passed (due to SSE optimization) /// we cannot convert it with current src_end, but we may have more /// bytes to convert and eventually got correct symbol. - if (partial && src_sequence_length > static_cast(src_end-src)) + if (partial && src_sequence_length > static_cast(src_end - src)) return false; auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_end - src); @@ -181,7 +181,9 @@ private: #ifdef __SSE2__ static constexpr auto bytes_sse = sizeof(__m128i); - const auto * src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse; + + /// If we are before this position, we can still read at least bytes_sse. + const auto * src_end_sse = src_end - bytes_sse + 1; /// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f) const auto v_zero = _mm_setzero_si128(); @@ -227,9 +229,11 @@ private: { /// UTF-8 + /// Find the offset of the next string after src size_t offset_from_begin = src - begin; while (offset_from_begin >= *offset_it) ++offset_it; + /// Do not allow one row influence another (since row may have invalid sequence, and break the next) const UInt8 * row_end = begin + *offset_it; chassert(row_end >= src); @@ -247,8 +251,9 @@ private: } } - /// Find which offset src has now - while (offset_it != offsets.end() && static_cast(src - begin) >= *offset_it) + /// Find the offset of the next string after src + size_t offset_from_begin = src - begin; + while (offset_it != offsets.end() && offset_from_begin >= *offset_it) ++offset_it; #endif From f9dec21c9b7324e9184851e66129f6c2fcf7d0b2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 06:01:37 +0200 Subject: [PATCH 110/171] Add a test --- tests/queries/0_stateless/02807_lower_utf8_msan.reference | 4 ++++ tests/queries/0_stateless/02807_lower_utf8_msan.sql | 2 ++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/02807_lower_utf8_msan.reference create mode 100644 tests/queries/0_stateless/02807_lower_utf8_msan.sql diff --git a/tests/queries/0_stateless/02807_lower_utf8_msan.reference b/tests/queries/0_stateless/02807_lower_utf8_msan.reference new file mode 100644 index 00000000000..7d7df6af777 --- /dev/null +++ b/tests/queries/0_stateless/02807_lower_utf8_msan.reference @@ -0,0 +1,4 @@ +©-------------------- +©-------------------------------------- +AAAAАБВГAAAAAAAAAAAAАБВГAAAAAAAA +AAAAАБВГAAAAAAAAAAAAАБВГAAAAAAAA diff --git a/tests/queries/0_stateless/02807_lower_utf8_msan.sql b/tests/queries/0_stateless/02807_lower_utf8_msan.sql new file mode 100644 index 00000000000..e9eb18bf615 --- /dev/null +++ b/tests/queries/0_stateless/02807_lower_utf8_msan.sql @@ -0,0 +1,2 @@ +SELECT lowerUTF8(arrayJoin(['©--------------------------------------', '©--------------------'])) ORDER BY 1; +SELECT upperUTF8(materialize('aaaaАБВГaaaaaaaaaaaaАБВГAAAAaaAA')) FROM numbers(2); From 511b2cdc21f6e0350aea15cb3e786633a9e52ba3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 06:10:57 +0200 Subject: [PATCH 111/171] Increase timeout of fast test --- tests/ci/fast_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 89066ade2cb..d377826513b 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -150,7 +150,7 @@ def main(): os.makedirs(logs_path) run_log_path = os.path.join(logs_path, "run.log") - with TeePopen(run_cmd, run_log_path, timeout=40 * 60) as process: + with TeePopen(run_cmd, run_log_path, timeout = 90 * 60) as process: retcode = process.wait() if retcode == 0: logging.info("Run successfully") From 4fd1fc4507c0628333d869cc21a757e7de7b6da7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 06:14:31 +0200 Subject: [PATCH 112/171] Fix typo --- tests/ci/tee_popen.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/tee_popen.py b/tests/ci/tee_popen.py index 4869301785e..7872b489951 100644 --- a/tests/ci/tee_popen.py +++ b/tests/ci/tee_popen.py @@ -11,7 +11,7 @@ import os import sys -# Very simple tee logic implementation. You can specify shell command, output +# Very simple tee logic implementation. You can specify a shell command, output # logfile and env variables. After TeePopen is created you can only wait until # it finishes. stderr and stdout will be redirected both to specified file and # stdout. From 8017ea79c173d2e14f76111fff97e0b1f84d934e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 06:54:00 +0200 Subject: [PATCH 113/171] Fix bad tests for DNS --- .../test.py | 18 ++++++++++++++++++ .../test.py | 17 +++++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/test.py b/tests/integration/test_host_regexp_multiple_ptr_records/test.py index ba807d3b5d7..82ae0b6c527 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records/test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records/test.py @@ -1,5 +1,7 @@ import pytest +import socket from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check +from time import sleep import os DOCKER_COMPOSE_PATH = get_docker_compose_path() @@ -32,11 +34,27 @@ def started_cluster(): cluster.shutdown() +def check_ptr_record(ip, hostname): + try: + host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip) + if hostname.lower() == host.lower(): + return True + except socket.herror: + pass + return False + + def setup_dns_server(ip): domains_string = "test3.example.com test2.example.com test1.example.com" example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) + # DNS server takes time to reload the configuration. + for try_num in range(10): + if all(check_ptr_record(ip, host) for host in domains_string.split()): + break + sleep(1) + def setup_ch_server(dns_server_ip): ch_server.exec_in_container( diff --git a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py index 62f47579612..d73e8813e79 100644 --- a/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py +++ b/tests/integration/test_host_regexp_multiple_ptr_records_concurrent/test.py @@ -1,4 +1,5 @@ import pytest +import socket from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check from time import sleep import os @@ -31,11 +32,27 @@ def started_cluster(): cluster.shutdown() +def check_ptr_record(ip, hostname): + try: + host, aliaslist, ipaddrlist = socket.gethostbyaddr(ip) + if hostname.lower() == host.lower(): + return True + except socket.herror: + pass + return False + + def setup_dns_server(ip): domains_string = "test3.example.com test2.example.com test1.example.com" example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) + # DNS server takes time to reload the configuration. + for try_num in range(10): + if all(check_ptr_record(ip, host) for host in domains_string.split()): + break + sleep(1) + def setup_ch_server(dns_server_ip): ch_server.exec_in_container( From c1da927b895e2982f729d90b20a212bd40581ea6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 07:01:36 +0200 Subject: [PATCH 114/171] Attempt to fix the `relax_too_many_parts` test --- tests/queries/0_stateless/02458_relax_too_many_parts.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02458_relax_too_many_parts.sql b/tests/queries/0_stateless/02458_relax_too_many_parts.sql index a1f8e86fce5..01f9edce675 100644 --- a/tests/queries/0_stateless/02458_relax_too_many_parts.sql +++ b/tests/queries/0_stateless/02458_relax_too_many_parts.sql @@ -1,5 +1,5 @@ DROP TABLE IF EXISTS test; -CREATE TABLE test (x UInt64, s String) ENGINE = MergeTree ORDER BY tuple() SETTINGS parts_to_throw_insert = 3; +CREATE TABLE test (x UInt64, s String) ENGINE = MergeTree ORDER BY tuple() SETTINGS parts_to_throw_insert = 3, max_parts_to_merge_at_once = 1; -- The "too many parts" threshold works: SET max_block_size = 1, min_insert_block_size_rows = 1, min_insert_block_size_bytes = 1; @@ -14,7 +14,7 @@ ALTER TABLE test MODIFY SETTING max_avg_part_size_for_too_many_parts = '1M'; -- It works in the same way if parts are small: SYSTEM START MERGES test; -OPTIMIZE TABLE test FINAL; +OPTIMIZE TABLE test FINAL SETTINGS optimize_throw_if_noop=1; SYSTEM STOP MERGES test; INSERT INTO test VALUES (5, 'a'); @@ -23,7 +23,7 @@ INSERT INTO test VALUES (7, 'a'); -- { serverError TOO_MANY_PARTS } -- But it allows having more parts if their average size is large: SYSTEM START MERGES test; -OPTIMIZE TABLE test FINAL; +OPTIMIZE TABLE test FINAL SETTINGS optimize_throw_if_noop=1; SYSTEM STOP MERGES test; SET max_block_size = 65000, min_insert_block_size_rows = 65000, min_insert_block_size_bytes = '1M'; From 266b08132cb4908123ee4479dfa3e93b3ca0212e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 07:11:41 +0200 Subject: [PATCH 115/171] Fix MySQL test in Debug mode --- .../02479_mysql_connect_to_self.sql | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql index ff46ebf1ed2..c5c5dfb8c4a 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql @@ -8,28 +8,28 @@ INSERT INTO foo VALUES (1, 'one', -1, 'een'), (2, 'two', -2, 'twee'), (3, 'three SET allow_experimental_analyzer = 1; SELECT '---'; -SELECT * FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') ORDER BY key; +SELECT * FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100) ORDER BY key; SELECT '---'; -SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', ''); +SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100); SELECT '---'; -SELECT 1 FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', ''); +SELECT 1 FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100); SELECT '---'; -SELECT key FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') ORDER BY key; +SELECT key FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100) ORDER BY key; SELECT '---'; -SELECT b, a FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') ORDER BY a; +SELECT b, a FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100) ORDER BY a; SELECT '---'; -SELECT b, a FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') ORDER BY c; +SELECT b, a FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100) ORDER BY c; SELECT '---'; -SELECT b FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') WHERE c != 'twee' ORDER BY b; +SELECT b FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100) WHERE c != 'twee' ORDER BY b; SELECT '---'; -SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '') WHERE c != 'twee'; +SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connect_timeout = 100, connection_wait_timeout = 100) WHERE c != 'twee'; EXPLAIN QUERY TREE dump_ast = 1 SELECT * FROM mysql( @@ -38,5 +38,5 @@ SELECT * FROM mysql( ); SELECT '---'; -SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connection_pool_size = 1); +SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connection_pool_size = 1, connect_timeout = 100, connection_wait_timeout = 100); SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connection_pool_size = 0); -- { serverError BAD_ARGUMENTS } From b46817cbded44f67d9d68421cfe103cb345ef622 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 07:18:29 +0200 Subject: [PATCH 116/171] Fix bad test `01018_Distributed__shard_num` --- tests/queries/0_stateless/01018_Distributed__shard_num.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01018_Distributed__shard_num.sql b/tests/queries/0_stateless/01018_Distributed__shard_num.sql index 7e31062348d..6f9d8a6a1bb 100644 --- a/tests/queries/0_stateless/01018_Distributed__shard_num.sql +++ b/tests/queries/0_stateless/01018_Distributed__shard_num.sql @@ -3,6 +3,9 @@ -- make the order static SET max_threads = 1; +-- data should be inserted into Distributed table synchronously +SET insert_distributed_sync = 1; + DROP TABLE IF EXISTS mem1; DROP TABLE IF EXISTS mem2; DROP TABLE IF EXISTS mem3; From 071cd58be805f2021fbead23fb08f2853d816b12 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 08:31:02 +0200 Subject: [PATCH 117/171] Fix "logical error" in addressToLineWithInlines --- src/Functions/addressToLineWithInlines.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/addressToLineWithInlines.cpp b/src/Functions/addressToLineWithInlines.cpp index 9a7b0858b87..43e2eeb10c7 100644 --- a/src/Functions/addressToLineWithInlines.cpp +++ b/src/Functions/addressToLineWithInlines.cpp @@ -59,14 +59,13 @@ protected: void setResult(StringRefs & result, const Dwarf::LocationInfo & location, const std::vector & inline_frames) const override { - appendLocationToResult(result, location, nullptr); for (const auto & inline_frame : inline_frames) appendLocationToResult(result, inline_frame.location, &inline_frame); } -private: - inline ALWAYS_INLINE void appendLocationToResult(StringRefs & result, const Dwarf::LocationInfo & location, const Dwarf::SymbolizedFrame * frame) const +private: + void appendLocationToResult(StringRefs & result, const Dwarf::LocationInfo & location, const Dwarf::SymbolizedFrame * frame) const { const char * arena_begin = nullptr; WriteBufferFromArena out(cache.arena, arena_begin); @@ -83,6 +82,7 @@ private: } result.emplace_back(out.complete()); + out.finalize(); } }; From f4f2c1476cc9484bd470328c65c454e7b8874169 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 08:40:11 +0200 Subject: [PATCH 118/171] Fix test 01280_ttl_where_group_by --- .../0_stateless/01280_ttl_where_group_by.sh | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.sh b/tests/queries/0_stateless/01280_ttl_where_group_by.sh index ebef8f2a797..e6f83d6edd1 100755 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.sh +++ b/tests/queries/0_stateless/01280_ttl_where_group_by.sh @@ -15,9 +15,11 @@ function optimize() done } +# "SETTINGS max_parts_to_merge_at_once = 1" prevents merges to start before our own OPTIMIZE FINAL + echo "ttl_01280_1" $CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_1 (a Int, b Int, x Int, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second delete where x % 10 == 0 and y > 5; +create table ttl_01280_1 (a Int, b Int, x Int, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second delete where x % 10 == 0 and y > 5 SETTINGS max_parts_to_merge_at_once = 1; insert into ttl_01280_1 values (1, 1, 0, 4, now() + 10); insert into ttl_01280_1 values (1, 1, 10, 6, now()); insert into ttl_01280_1 values (1, 2, 3, 7, now()); @@ -34,7 +36,7 @@ $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_2" echo "ttl_01280_2" $CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_2 (a Int, b Int, x Array(Int32), y Double, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b set x = minForEach(x), y = sum(y), d = max(d); +create table ttl_01280_2 (a Int, b Int, x Array(Int32), y Double, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b set x = minForEach(x), y = sum(y), d = max(d) SETTINGS max_parts_to_merge_at_once = 1; insert into ttl_01280_2 values (1, 1, array(0, 2, 3), 4, now() + 10); insert into ttl_01280_2 values (1, 1, array(5, 4, 3), 6, now()); insert into ttl_01280_2 values (1, 1, array(5, 5, 1), 7, now()); @@ -53,7 +55,7 @@ $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_3" echo "ttl_01280_3" $CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_3 (a Int, b Int, x Int64, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set b = min(b), x = argMax(x, d), y = argMax(y, d), d = max(d); +create table ttl_01280_3 (a Int, b Int, x Int64, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set b = min(b), x = argMax(x, d), y = argMax(y, d), d = max(d) SETTINGS max_parts_to_merge_at_once = 1; insert into ttl_01280_3 values (1, 1, 0, 4, now() + 10); insert into ttl_01280_3 values (1, 1, 10, 6, now() + 1); insert into ttl_01280_3 values (1, 2, 3, 7, now()); @@ -72,7 +74,7 @@ $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_4" echo "ttl_01280_4" $CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_4 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), -(a + b)) ttl d + interval 1 second group by toDate(d) set x = sum(x), y = max(y); +create table ttl_01280_4 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), -(a + b)) ttl d + interval 1 second group by toDate(d) set x = sum(x), y = max(y) SETTINGS max_parts_to_merge_at_once = 1; insert into ttl_01280_4 values (1, 1, 0, 4, now() + 10); insert into ttl_01280_4 values (10, 2, 3, 3, now()); insert into ttl_01280_4 values (2, 10, 1, 7, now()); @@ -86,7 +88,7 @@ $CLICKHOUSE_CLIENT --query "select x, y from ttl_01280_4 ORDER BY a, b, x, y" $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_5" echo "ttl_01280_5" -$CLICKHOUSE_CLIENT -n --query "create table ttl_01280_5 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a set x = sum(x), b = argMax(b, -b); +$CLICKHOUSE_CLIENT -n --query "create table ttl_01280_5 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a set x = sum(x), b = argMax(b, -b) SETTINGS max_parts_to_merge_at_once = 1; insert into ttl_01280_5 values (1, 2, 3, 5, now()); insert into ttl_01280_5 values (2, 10, 1, 5, now()); insert into ttl_01280_5 values (2, 3, 5, 5, now()); @@ -100,7 +102,7 @@ $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_6" echo "ttl_01280_6" $CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_6 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a; +create table ttl_01280_6 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a SETTINGS max_parts_to_merge_at_once = 1; insert into ttl_01280_6 values (1, 2, 3, 5, now()); insert into ttl_01280_6 values (2, 10, 3, 5, now()); insert into ttl_01280_6 values (2, 3, 3, 5, now()); From b3f84a2b63cc2e516473d11c3d8852416454641c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 08:48:31 +0200 Subject: [PATCH 119/171] Maybe fix the "many mutations" test --- tests/queries/0_stateless/02125_many_mutations.sh | 6 ++++-- tests/queries/0_stateless/02125_many_mutations_2.sh | 6 ++++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02125_many_mutations.sh b/tests/queries/0_stateless/02125_many_mutations.sh index c3108df5ae3..b42d5bb15d3 100755 --- a/tests/queries/0_stateless/02125_many_mutations.sh +++ b/tests/queries/0_stateless/02125_many_mutations.sh @@ -5,7 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x settings number_of_mutations_to_delay = 0, number_of_mutations_to_throw = 0" +# "max_parts_to_merge_at_once = 1" prevents merges to start in background before our own OPTIMIZE FINAL + +$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x settings number_of_mutations_to_delay = 0, number_of_mutations_to_throw = 0, max_parts_to_merge_at_once = 1" $CLICKHOUSE_CLIENT -q "insert into many_mutations values (0, 0), (1, 1)" $CLICKHOUSE_CLIENT -q "system stop merges many_mutations" @@ -44,6 +46,6 @@ wait $CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" $CLICKHOUSE_CLIENT -q "system start merges many_mutations" -$CLICKHOUSE_CLIENT -q "optimize table many_mutations final" +$CLICKHOUSE_CLIENT -q "optimize table many_mutations final" --optimize_throw_if_noop 1 $CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" $CLICKHOUSE_CLIENT -q "select x, y from many_mutations order by x" diff --git a/tests/queries/0_stateless/02125_many_mutations_2.sh b/tests/queries/0_stateless/02125_many_mutations_2.sh index 52866a54974..e5e3070a944 100755 --- a/tests/queries/0_stateless/02125_many_mutations_2.sh +++ b/tests/queries/0_stateless/02125_many_mutations_2.sh @@ -5,7 +5,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x settings number_of_mutations_to_delay = 0, number_of_mutations_to_throw = 0" +# "max_parts_to_merge_at_once = 1" prevents merges to start in background before our own OPTIMIZE FINAL + +$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x settings number_of_mutations_to_delay = 0, number_of_mutations_to_throw = 0, max_parts_to_merge_at_once = 1" $CLICKHOUSE_CLIENT -q "insert into many_mutations select number, number + 1 from numbers(2000)" $CLICKHOUSE_CLIENT -q "system stop merges many_mutations" @@ -44,7 +46,7 @@ wait $CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" $CLICKHOUSE_CLIENT -q "system start merges many_mutations" -$CLICKHOUSE_CLIENT -q "optimize table many_mutations final" +$CLICKHOUSE_CLIENT -q "optimize table many_mutations final" --optimize_throw_if_noop 1 $CLICKHOUSE_CLIENT -q "system flush logs" $CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" $CLICKHOUSE_CLIENT -q "select count() from many_mutations" From e9a9a019cf10c2cbd13a28084c4d02c6114a71ad Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 09:28:15 +0200 Subject: [PATCH 120/171] Attempt to fix test_ssl_cert_authentication --- .../test_ssl_cert_authentication/configs/ssl_config.xml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml b/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml index ed3b2b595db..d0b58d984bf 100644 --- a/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml +++ b/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml @@ -18,6 +18,10 @@ /etc/clickhouse-server/config.d/server-key.pem /etc/clickhouse-server/config.d/ca-cert.pem relaxed + + + RejectCertificateHandler + @@ -25,12 +29,9 @@ true sslv2,sslv3 true - - RejectCertificateHandler - \ No newline at end of file From c4ea7ab5b15add3d3b69412224f9c53e30be1661 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Jun 2023 09:31:09 +0200 Subject: [PATCH 121/171] Attempt to fix test_ssl_cert_authentication --- programs/server/config.xml | 7 +++++++ .../test_ssl_cert_authentication/configs/ssl_config.xml | 4 ---- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/programs/server/config.xml b/programs/server/config.xml index 2f69f23a718..acd6d92a896 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -282,6 +282,13 @@ true sslv2,sslv3 true + + + + RejectCertificateHandler + diff --git a/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml b/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml index d0b58d984bf..24c9eb8891f 100644 --- a/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml +++ b/tests/integration/test_ssl_cert_authentication/configs/ssl_config.xml @@ -18,10 +18,6 @@ /etc/clickhouse-server/config.d/server-key.pem /etc/clickhouse-server/config.d/ca-cert.pem relaxed - - - RejectCertificateHandler - From 5871ca0836e9a21790f6bf3cff2d3134523f3a08 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 24 Jun 2023 20:38:03 +0200 Subject: [PATCH 122/171] Fix performance tests due to warnings from jemalloc about Per-CPU arena disabled jemalloc can show the following warning: Number of CPUs detected is not deterministic. Per-CPU arena disabled It will be shown if one of the following returns different number of CPUs: - _SC_NPROCESSORS_ONLN - _SC_NPROCESSORS_CONF - sched_getaffinity() And actually for my CPU linux returns different numbers, because there are more possible CPUs then online, from dmesg: smpboot: Allowing 128 CPUs, 64 hotplug CPUs And from sysfs: # grep . /sys/devices/system/cpu/{possible,online,offline} /sys/devices/system/cpu/possible:0-127 /sys/devices/system/cpu/online:0-63 /sys/devices/system/cpu/offline:64-127 From ACPI: # acpidump -o acpi # acpixtract -a acpi # iasl -d *.dat # grep -e 'Processor Enabled' apic.dsl | sort | uniq -c 64 Processor Enabled : 0 64 Processor Enabled : 1 So I guess this is the same as what happened in this perf run [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/51360/5d43a64112711b339b82b1c0e8df7882546a1a3c/performance_comparison_[4_4]/report.html P.S. personally I, just use cmdline=possible_cpus=64 to fix this for my setup. Signed-off-by: Azat Khuzhin --- docker/test/performance-comparison/compare.sh | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 293ad9ac411..798d2a40b12 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -14,6 +14,13 @@ LEFT_SERVER_PORT=9001 # patched version RIGHT_SERVER_PORT=9002 +# abort_conf -- abort if some options is not recognized +# abort -- abort if something is not right in the env (i.e. per-cpu arenas does not work) +# narenas -- set them explicitly to avoid disabling per-cpu arena in env +# that returns different number of CPUs for some of the following +# _SC_NPROCESSORS_ONLN/_SC_NPROCESSORS_CONF/sched_getaffinity +export MALLOC_CONF="abort_conf:true,abort:true,narenas:$(nproc --all)" + function wait_for_server # port, pid { for _ in {1..60} @@ -109,10 +116,6 @@ function restart while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done echo all killed - # Change the jemalloc settings here. - # https://github.com/jemalloc/jemalloc/wiki/Getting-Started - export MALLOC_CONF="confirm_conf:true" - set -m # Spawn servers in their own process groups local left_server_opts=( @@ -147,8 +150,6 @@ function restart set +m - unset MALLOC_CONF - wait_for_server $LEFT_SERVER_PORT $left_pid echo left ok From 477b707ff1765d8a2d62ad21b869d544b212de96 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 02:02:24 +0200 Subject: [PATCH 123/171] Revert "Merge pull request #50951 from ZhiguoZh/20230607-toyear-fix" This reverts commit 6bbd0d144df01b07a28d3d9927ce2a6c1dc2ee56, reversing changes made to 74cb79769bbaa0c4619ca7cb382e6e37c8c7d7b5. --- src/Functions/DateTimeTransforms.h | 72 ------- .../FunctionDateOrDateTimeToSomething.h | 13 -- src/Functions/IFunction.h | 24 +-- src/Functions/IFunctionAdaptors.h | 7 - ...OrDateTimeConverterWithPreimageVisitor.cpp | 199 ------------------ ...teOrDateTimeConverterWithPreimageVisitor.h | 37 ---- src/Interpreters/TreeOptimizer.cpp | 19 -- ...783_date_predicate_optimizations.reference | 52 ----- .../02783_date_predicate_optimizations.sql | 76 ------- ...dicate_optimizations_ast_rewrite.reference | 87 -------- ...te_predicate_optimizations_ast_rewrite.sql | 47 ----- 11 files changed, 1 insertion(+), 632 deletions(-) delete mode 100644 src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp delete mode 100644 src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h delete mode 100644 tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference delete mode 100644 tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index 84c71c89b11..019e0c42cde 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -322,7 +322,6 @@ struct ToTimeImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToDateImpl; }; @@ -394,7 +393,6 @@ struct ToStartOfSecondImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -442,7 +440,6 @@ struct ToStartOfMillisecondImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -486,7 +483,6 @@ struct ToStartOfMicrosecondImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -524,7 +520,6 @@ struct ToStartOfNanosecondImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -723,28 +718,6 @@ struct ToYearImpl return time_zone.toYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return true; } - - static inline RangeOrNull getPreimage(const IDataType & type, const Field & point) - { - if (point.getType() != Field::Types::UInt64) return std::nullopt; - - auto year = point.get(); - if (year < DATE_LUT_MIN_YEAR || year >= DATE_LUT_MAX_YEAR) return std::nullopt; - - const DateLUTImpl & date_lut = DateLUT::instance(); - - auto start_time = date_lut.makeDateTime(year, 1, 1, 0, 0, 0); - auto end_time = date_lut.addYears(start_time, 1); - - if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type)) - return {std::make_pair(Field(start_time), Field(end_time))}; - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64", - type.getName(), name); - } - using FactorTransform = ZeroTransform; }; @@ -818,7 +791,6 @@ struct ToQuarterImpl { return time_zone.toQuarter(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -843,7 +815,6 @@ struct ToMonthImpl { return time_zone.toMonth(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -869,7 +840,6 @@ struct ToDayOfMonthImpl return time_zone.toDayOfMonth(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfMonthImpl; }; @@ -917,7 +887,6 @@ struct ToDayOfYearImpl { return time_zone.toDayOfYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfYearImpl; }; @@ -942,7 +911,6 @@ struct ToHourImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToDateImpl; }; @@ -971,7 +939,6 @@ struct TimezoneOffsetImpl throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToTimeImpl; }; @@ -995,7 +962,6 @@ struct ToMinuteImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfHourImpl; }; @@ -1020,7 +986,6 @@ struct ToSecondImpl { throwDateTimeIsNotSupported(name); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToStartOfMinuteImpl; }; @@ -1045,7 +1010,6 @@ struct ToISOYearImpl { return time_zone.toISOYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1102,7 +1066,6 @@ struct ToISOWeekImpl { return time_zone.toISOWeek(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ToISOYearImpl; }; @@ -1145,7 +1108,6 @@ struct ToRelativeYearNumImpl { return time_zone.toYear(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1177,7 +1139,6 @@ struct ToRelativeQuarterNumImpl { return time_zone.toRelativeQuarterNum(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1209,7 +1170,6 @@ struct ToRelativeMonthNumImpl { return time_zone.toRelativeMonthNum(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1241,7 +1201,6 @@ struct ToRelativeWeekNumImpl { return time_zone.toRelativeWeekNum(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1273,7 +1232,6 @@ struct ToRelativeDayNumImpl { return static_cast(d); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1311,7 +1269,6 @@ struct ToRelativeHourNumImpl else return static_cast(time_zone.toRelativeHourNum(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1343,7 +1300,6 @@ struct ToRelativeMinuteNumImpl { return static_cast(time_zone.toRelativeMinuteNum(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1372,7 +1328,6 @@ struct ToRelativeSecondNumImpl { return static_cast(time_zone.fromDayNum(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1397,31 +1352,6 @@ struct ToYYYYMMImpl { return time_zone.toNumYYYYMM(DayNum(d)); } - static inline constexpr bool hasPreimage() { return true; } - - static inline RangeOrNull getPreimage(const IDataType & type, const Field & point) - { - if (point.getType() != Field::Types::UInt64) return std::nullopt; - - auto year_month = point.get(); - auto year = year_month / 100; - auto month = year_month % 100; - - if (year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || (year == DATE_LUT_MAX_YEAR && month == 12)) - return std::nullopt; - - const DateLUTImpl & date_lut = DateLUT::instance(); - - auto start_time = date_lut.makeDateTime(year, month, 1, 0, 0, 0); - auto end_time = date_lut.addMonths(start_time, 1); - - if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type)) - return {std::make_pair(Field(start_time), Field(end_time))}; - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64", - type.getName(), name); - } using FactorTransform = ZeroTransform; }; @@ -1446,7 +1376,6 @@ struct ToYYYYMMDDImpl { return time_zone.toNumYYYYMMDD(DayNum(d)); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; @@ -1471,7 +1400,6 @@ struct ToYYYYMMDDhhmmssImpl { return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(DayNum(d))); } - static inline constexpr bool hasPreimage() { return false; } using FactorTransform = ZeroTransform; }; diff --git a/src/Functions/FunctionDateOrDateTimeToSomething.h b/src/Functions/FunctionDateOrDateTimeToSomething.h index d98b788c7d7..82818cc3d2b 100644 --- a/src/Functions/FunctionDateOrDateTimeToSomething.h +++ b/src/Functions/FunctionDateOrDateTimeToSomething.h @@ -7,7 +7,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NOT_IMPLEMENTED; } /// See DateTimeTransforms.h @@ -84,18 +83,6 @@ public: arguments[0].type->getName(), this->getName()); } - bool hasInformationAboutPreimage() const override { return Transform::hasPreimage(); } - - RangeOrNull getPreimage(const IDataType & type, const Field & point) const override - { - if constexpr (Transform::hasPreimage()) - return Transform::getPreimage(type, point); - else - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "Function {} has no information about its preimage", - Transform::name); - } - }; } diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 433cb61d04e..1e4f8bf1102 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -2,8 +2,6 @@ #include #include -#include -#include #include #include #include @@ -32,8 +30,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } -/// A left-closed and right-open interval representing the preimage of a function. -using RangeOrNull = std::optional>; +class Field; /// The simplest executable object. /// Motivation: @@ -231,12 +228,6 @@ public: */ virtual bool hasInformationAboutMonotonicity() const { return false; } - /** Lets you know if the function has its definition of preimage. - * This is used to work with predicate optimizations, where the comparison between - * f(x) and a constant c could be converted to the comparison between x and f's preimage [b, e). - */ - virtual bool hasInformationAboutPreimage() const { return false; } - struct ShortCircuitSettings { /// Should we enable lazy execution for the first argument of short-circuit function? @@ -290,14 +281,6 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName()); } - /** Get the preimage of a function in the form of a left-closed and right-open interval. Call only if hasInformationAboutPreimage. - * std::nullopt might be returned if the point (a single value) is invalid for this function. - */ - virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName()); - } - }; using FunctionBasePtr = std::shared_ptr; @@ -487,17 +470,12 @@ public: virtual bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const = 0; virtual bool hasInformationAboutMonotonicity() const { return false; } - virtual bool hasInformationAboutPreimage() const { return false; } using Monotonicity = IFunctionBase::Monotonicity; virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName()); } - virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName()); - } /// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored). virtual size_t getNumberOfArguments() const = 0; diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 123fdbc2f50..23725b1a8b1 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -90,17 +90,10 @@ public: bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); } - bool hasInformationAboutPreimage() const override { return function->hasInformationAboutPreimage(); } - Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override { return function->getMonotonicityForRange(type, left, right); } - - RangeOrNull getPreimage(const IDataType & type, const Field & point) const override - { - return function->getPreimage(type, point); - } private: std::shared_ptr function; DataTypes arguments; diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp deleted file mode 100644 index a377bb4bba6..00000000000 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.cpp +++ /dev/null @@ -1,199 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -/** Given a monotonic non-decreasing function f(x), which satisfies f(x) = c for any value x within [b, e). - * We could convert it into its equivalent form, x >= b AND x < e, which is free from the invocation of the function. - * And we could apply the similar transformation to other comparisons. The suggested transformations list: - * - * f(x) == c -> x >= b AND x < e - * f(x) != c -> x < b OR x >= e - * f(x) > c -> x >= e - * f(x) >= c -> x >= b - * f(x) < c -> x < b - * f(x) <= c -> x < e - * - * This function generates a new AST with the transformed relation. - */ -ASTPtr generateOptimizedDateFilterAST(const String & comparator, const NameAndTypePair & column, const std::pair& range) -{ - const DateLUTImpl & date_lut = DateLUT::instance(); - - const String & column_name = column.name; - String start_date_or_date_time; - String end_date_or_date_time; - - if (isDateOrDate32(column.type.get())) - { - start_date_or_date_time = date_lut.dateToString(range.first.get()); - end_date_or_date_time = date_lut.dateToString(range.second.get()); - } - else if (isDateTime(column.type.get()) || isDateTime64(column.type.get())) - { - start_date_or_date_time = date_lut.timeToString(range.first.get()); - end_date_or_date_time = date_lut.timeToString(range.second.get()); - } - else [[unlikely]] return {}; - - if (comparator == "equals") - { - return makeASTFunction("and", - makeASTFunction("greaterOrEquals", - std::make_shared(column_name), - std::make_shared(start_date_or_date_time) - ), - makeASTFunction("less", - std::make_shared(column_name), - std::make_shared(end_date_or_date_time) - ) - ); - } - else if (comparator == "notEquals") - { - return makeASTFunction("or", - makeASTFunction("less", - std::make_shared(column_name), - std::make_shared(start_date_or_date_time) - ), - makeASTFunction("greaterOrEquals", - std::make_shared(column_name), - std::make_shared(end_date_or_date_time) - ) - ); - } - else if (comparator == "greater") - { - return makeASTFunction("greaterOrEquals", - std::make_shared(column_name), - std::make_shared(end_date_or_date_time) - ); - } - else if (comparator == "lessOrEquals") - { - return makeASTFunction("less", - std::make_shared(column_name), - std::make_shared(end_date_or_date_time) - ); - } - else if (comparator == "less" || comparator == "greaterOrEquals") - { - return makeASTFunction(comparator, - std::make_shared(column_name), - std::make_shared(start_date_or_date_time) - ); - } - else [[unlikely]] - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected equals, notEquals, less, lessOrEquals, greater, greaterOrEquals. Actual {}", - comparator); - } -} - -void OptimizeDateOrDateTimeConverterWithPreimageMatcher::visit(const ASTFunction & function, ASTPtr & ast, const Data & data) -{ - const static std::unordered_map swap_relations = { - {"equals", "equals"}, - {"notEquals", "notEquals"}, - {"less", "greater"}, - {"greater", "less"}, - {"lessOrEquals", "greaterOrEquals"}, - {"greaterOrEquals", "lessOrEquals"}, - }; - - if (!swap_relations.contains(function.name)) return; - - if (!function.arguments || function.arguments->children.size() != 2) return; - - size_t func_id = function.arguments->children.size(); - - for (size_t i = 0; i < function.arguments->children.size(); i++) - { - if (const auto * func = function.arguments->children[i]->as()) - { - func_id = i; - } - } - - if (func_id == function.arguments->children.size()) return; - - size_t literal_id = 1 - func_id; - const auto * literal = function.arguments->children[literal_id]->as(); - - if (!literal || literal->value.getType() != Field::Types::UInt64) return; - - String comparator = literal_id > func_id ? function.name : swap_relations.at(function.name); - - const auto * ast_func = function.arguments->children[func_id]->as(); - /// Currently we only handle single-argument functions. - if (!ast_func || !ast_func->arguments || ast_func->arguments->children.size() != 1) return; - - const auto * column_id = ast_func->arguments->children.at(0)->as(); - if (!column_id) return; - - auto pos = IdentifierSemantic::getMembership(*column_id); - if (!pos) - pos = IdentifierSemantic::chooseTableColumnMatch(*column_id, data.tables, true); - if (!pos) - return; - - if (*pos >= data.tables.size()) - return; - - auto data_type_and_name = data.tables[*pos].columns.tryGetByName(column_id->shortName()); - if (!data_type_and_name) return; - - const auto & converter = FunctionFactory::instance().tryGet(ast_func->name, data.context); - if (!converter) return; - - ColumnsWithTypeAndName args; - args.emplace_back(data_type_and_name->type, "tmp"); - auto converter_base = converter->build(args); - if (!converter_base || !converter_base->hasInformationAboutPreimage()) return; - - auto preimage_range = converter_base->getPreimage(*(data_type_and_name->type), literal->value); - if (!preimage_range) return; - - const auto new_ast = generateOptimizedDateFilterAST(comparator, *data_type_and_name, *preimage_range); - if (!new_ast) return; - - ast = new_ast; -} - -bool OptimizeDateOrDateTimeConverterWithPreimageMatcher::needChildVisit(ASTPtr & ast, ASTPtr & /*child*/) -{ - const static std::unordered_set relations = { - "equals", - "notEquals", - "less", - "greater", - "lessOrEquals", - "greaterOrEquals", - }; - - if (const auto * ast_function = ast->as()) - { - return !relations.contains(ast_function->name); - } - - return true; -} - -} diff --git a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h b/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h deleted file mode 100644 index 778fa462364..00000000000 --- a/src/Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -class ASTFunction; - -/** Replace predicate having Date/DateTime converters with their preimages to improve performance. - * Given a Date column c, toYear(c) = 2023 -> c >= '2023-01-01' AND c < '2024-01-01' - * Or if c is a DateTime column, toYear(c) = 2023 -> c >= '2023-01-01 00:00:00' AND c < '2024-01-01 00:00:00'. - * The similar optimization also applies to other converters. - */ -class OptimizeDateOrDateTimeConverterWithPreimageMatcher -{ -public: - struct Data - { - const TablesWithColumns & tables; - ContextPtr context; - }; - - static void visit(ASTPtr & ast, Data & data) - { - if (const auto * ast_function = ast->as()) - visit(*ast_function, ast, data); - } - - static void visit(const ASTFunction & function, ASTPtr & ast, const Data & data); - - static bool needChildVisit(ASTPtr & ast, ASTPtr & child); -}; - -using OptimizeDateOrDateTimeConverterWithPreimageVisitor = InDepthNodeVisitor; -} diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index fd4d2c9d846..c38b3c79026 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include @@ -678,21 +677,6 @@ void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, ContextPtr context) RemoveInjectiveFunctionsVisitor(data).visit(query); } -void optimizeDateFilters(ASTSelectQuery * select_query, const std::vector & tables_with_columns, ContextPtr context) -{ - /// Predicates in HAVING clause has been moved to WHERE clause. - if (select_query->where()) - { - OptimizeDateOrDateTimeConverterWithPreimageVisitor::Data data{tables_with_columns, context}; - OptimizeDateOrDateTimeConverterWithPreimageVisitor(data).visit(select_query->refWhere()); - } - if (select_query->prewhere()) - { - OptimizeDateOrDateTimeConverterWithPreimageVisitor::Data data{tables_with_columns, context}; - OptimizeDateOrDateTimeConverterWithPreimageVisitor(data).visit(select_query->refPrewhere()); - } -} - void transformIfStringsIntoEnum(ASTPtr & query) { std::unordered_set function_names = {"if", "transform"}; @@ -796,9 +780,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, tables_with_columns, result.storage_snapshot->metadata, result.storage); } - /// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, etc. - optimizeDateFilters(select_query, tables_with_columns, context); - /// GROUP BY injective function elimination. optimizeGroupBy(select_query, context); diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.reference b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference index 872a5dd1d7d..cd689b93034 100644 --- a/tests/queries/0_stateless/02783_date_predicate_optimizations.reference +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.reference @@ -1,54 +1,2 @@ 2021-12-31 23:00:00 0 2021-12-31 23:00:00 0 -Date -2 -3 -2 -4 -1 -3 -3 -2 -1 -4 -1 -4 -DateTime -2 -3 -2 -4 -1 -3 -3 -2 -1 -4 -1 -4 -Date32 -2 -3 -2 -4 -1 -3 -3 -2 -1 -4 -1 -4 -DateTime64 -2 -3 -2 -4 -1 -3 -3 -2 -1 -4 -1 -4 diff --git a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql index 0a2fa6cc93b..abb13f1005e 100644 --- a/tests/queries/0_stateless/02783_date_predicate_optimizations.sql +++ b/tests/queries/0_stateless/02783_date_predicate_optimizations.sql @@ -11,79 +11,3 @@ INSERT INTO source values ('2021-12-31 23:00:00', 0); SELECT * FROM source WHERE toYYYYMM(ts) = 202112; SELECT * FROM source WHERE toYear(ts) = 2021; - -DROP TABLE IF EXISTS source; -CREATE TABLE source -( - `dt` Date, - `ts` DateTime, - `dt_32` Date32, - `ts_64` DateTime64(3), - `n` Int32 -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(ts) -ORDER BY tuple(); - -INSERT INTO source values ('2022-12-31', '2022-12-31 23:59:59', '2022-12-31', '2022-12-31 23:59:59.123', 0); -INSERT INTO source values ('2023-01-01', '2023-01-01 00:00:00', '2023-01-01', '2023-01-01 00:00:00.000', 1); -INSERT INTO source values ('2023-12-01', '2023-12-01 00:00:00', '2023-12-01', '2023-12-01 00:00:00.000', 2); -INSERT INTO source values ('2023-12-31', '2023-12-31 23:59:59', '2023-12-31', '2023-12-31 23:59:59.123', 3); -INSERT INTO source values ('2024-01-01', '2024-01-01 00:00:00', '2024-01-01', '2024-01-01 00:00:00.000', 4); - -SELECT 'Date'; -SELECT count(*) FROM source WHERE toYYYYMM(dt) = 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt) <> 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt) < 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt) <= 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt) > 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt) >= 202312; -SELECT count(*) FROM source WHERE toYear(dt) = 2023; -SELECT count(*) FROM source WHERE toYear(dt) <> 2023; -SELECT count(*) FROM source WHERE toYear(dt) < 2023; -SELECT count(*) FROM source WHERE toYear(dt) <= 2023; -SELECT count(*) FROM source WHERE toYear(dt) > 2023; -SELECT count(*) FROM source WHERE toYear(dt) >= 2023; - -SELECT 'DateTime'; -SELECT count(*) FROM source WHERE toYYYYMM(ts) = 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts) <> 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts) < 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts) <= 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts) > 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts) >= 202312; -SELECT count(*) FROM source WHERE toYear(ts) = 2023; -SELECT count(*) FROM source WHERE toYear(ts) <> 2023; -SELECT count(*) FROM source WHERE toYear(ts) < 2023; -SELECT count(*) FROM source WHERE toYear(ts) <= 2023; -SELECT count(*) FROM source WHERE toYear(ts) > 2023; -SELECT count(*) FROM source WHERE toYear(ts) >= 2023; - -SELECT 'Date32'; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) = 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <> 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) < 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) <= 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) > 202312; -SELECT count(*) FROM source WHERE toYYYYMM(dt_32) >= 202312; -SELECT count(*) FROM source WHERE toYear(dt_32) = 2023; -SELECT count(*) FROM source WHERE toYear(dt_32) <> 2023; -SELECT count(*) FROM source WHERE toYear(dt_32) < 2023; -SELECT count(*) FROM source WHERE toYear(dt_32) <= 2023; -SELECT count(*) FROM source WHERE toYear(dt_32) > 2023; -SELECT count(*) FROM source WHERE toYear(dt_32) >= 2023; - -SELECT 'DateTime64'; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) = 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <> 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) < 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) <= 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) > 202312; -SELECT count(*) FROM source WHERE toYYYYMM(ts_64) >= 202312; -SELECT count(*) FROM source WHERE toYear(ts_64) = 2023; -SELECT count(*) FROM source WHERE toYear(ts_64) <> 2023; -SELECT count(*) FROM source WHERE toYear(ts_64) < 2023; -SELECT count(*) FROM source WHERE toYear(ts_64) <= 2023; -SELECT count(*) FROM source WHERE toYear(ts_64) > 2023; -SELECT count(*) FROM source WHERE toYear(ts_64) >= 2023; -DROP TABLE source; diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference deleted file mode 100644 index 9235e7e106a..00000000000 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.reference +++ /dev/null @@ -1,87 +0,0 @@ -SELECT value1 -FROM date_t -WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 < \'1993-01-01\') OR (date1 >= \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 >= \'1994-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 < \'1994-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 >= \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1998-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) OR ((date1 >= \'1994-01-01\') AND (date1 < \'1995-01-01\'))) AND ((id >= 1) AND (id <= 3)) -SELECT - value1, - toYear(date1) AS year1 -FROM date_t -WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 < \'1993-01-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -PREWHERE (date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\') -WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((id >= 1) AND (id <= 3)) AND ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) -SELECT value1 -FROM date_t -WHERE (toYYYYMM(date1) = 199300) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (toYYYYMM(date1) = 199313) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 >= \'1993-12-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 >= \'1992-03-01\') AND (date1 < \'1992-04-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 < \'1992-03-01\') OR (date1 >= \'1992-04-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 < \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 >= \'1992-04-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 < \'1992-04-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE (date1 >= \'1992-03-01\') AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date_t -WHERE ((date1 >= \'1992-03-01\') OR ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\'))) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM datetime_t -WHERE ((date1 >= \'1993-01-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM datetime_t -WHERE ((date1 >= \'1993-12-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date32_t -WHERE ((date1 >= \'1993-01-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM date32_t -WHERE ((date1 >= \'1993-12-01\') AND (date1 < \'1994-01-01\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM datetime64_t -WHERE ((date1 >= \'1993-01-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) -SELECT value1 -FROM datetime64_t -WHERE ((date1 >= \'1993-12-01 00:00:00\') AND (date1 < \'1994-01-01 00:00:00\')) AND ((id >= 1) AND (id <= 3)) diff --git a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql b/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql deleted file mode 100644 index 266be59b0a3..00000000000 --- a/tests/queries/0_stateless/02785_date_predicate_optimizations_ast_rewrite.sql +++ /dev/null @@ -1,47 +0,0 @@ -DROP TABLE IF EXISTS date_t; -CREATE TABLE date_t (id UInt32, value1 String, date1 Date) ENGINE ReplacingMergeTree() ORDER BY id; - -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) <> 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) < 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) > 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) <= 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) >= 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYear(date1) BETWEEN 1993 AND 1997 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE (toYear(date1) = 1993 OR toYear(date1) = 1994) AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1, toYear(date1) as year1 FROM date_t WHERE year1 = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE 1993 > toYear(date1) AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t PREWHERE toYear(date1) = 1993 WHERE id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE id BETWEEN 1 AND 3 HAVING toYear(date1) = 1993; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199300 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199313 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) = 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) <> 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) < 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) > 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) <= 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE toYYYYMM(date1) >= 199203 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date_t WHERE (toYYYYMM(date1) >= 199203 OR toYear(date1) = 1993) AND id BETWEEN 1 AND 3; -DROP TABLE date_t; - -DROP TABLE IF EXISTS datetime_t; -CREATE TABLE datetime_t (id UInt32, value1 String, date1 Datetime) ENGINE ReplacingMergeTree() ORDER BY id; - -EXPLAIN SYNTAX SELECT value1 FROM datetime_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM datetime_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -DROP TABLE datetime_t; - -DROP TABLE IF EXISTS date32_t; -CREATE TABLE date32_t (id UInt32, value1 String, date1 Date32) ENGINE ReplacingMergeTree() ORDER BY id; - -EXPLAIN SYNTAX SELECT value1 FROM date32_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM date32_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -DROP TABLE date32_t; - -DROP TABLE IF EXISTS datetime64_t; -CREATE TABLE datetime64_t (id UInt32, value1 String, date1 Datetime64) ENGINE ReplacingMergeTree() ORDER BY id; - -EXPLAIN SYNTAX SELECT value1 FROM datetime64_t WHERE toYear(date1) = 1993 AND id BETWEEN 1 AND 3; -EXPLAIN SYNTAX SELECT value1 FROM datetime64_t WHERE toYYYYMM(date1) = 199312 AND id BETWEEN 1 AND 3; -DROP TABLE datetime64_t; From 4ee74ec213b333893ddeb89331970e0d8758adbd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 02:19:52 +0200 Subject: [PATCH 124/171] Two tests are twice longer in average with Analyzer and sometimes failing --- docker/test/util/process_functional_tests_result.py | 4 ++-- tests/{broken_tests.txt => analyzer_tech_debt.txt} | 2 ++ tests/ci/functional_test_check.py | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) rename tests/{broken_tests.txt => analyzer_tech_debt.txt} (97%) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index c75a3500831..fd4cc9f4bf7 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -86,7 +86,7 @@ def process_test_log(log_path, broken_tests): test_name, "NOT_FAILED", test_time, - ["This test passed. Update broken_tests.txt.\n"], + ["This test passed. Update analyzer_tech_debt.txt.\n"], ) ) else: @@ -205,7 +205,7 @@ if __name__ == "__main__": parser.add_argument("--in-results-dir", default="/test_output/") parser.add_argument("--out-results-file", default="/test_output/test_results.tsv") parser.add_argument("--out-status-file", default="/test_output/check_status.tsv") - parser.add_argument("--broken-tests", default="/broken_tests.txt") + parser.add_argument("--broken-tests", default="/analyzer_tech_debt.txt") args = parser.parse_args() broken_tests = list() diff --git a/tests/broken_tests.txt b/tests/analyzer_tech_debt.txt similarity index 97% rename from tests/broken_tests.txt rename to tests/analyzer_tech_debt.txt index b3668b06e21..0872033aed0 100644 --- a/tests/broken_tests.txt +++ b/tests/analyzer_tech_debt.txt @@ -124,3 +124,5 @@ 02534_s3_cluster_insert_select_schema_inference 02765_parallel_replicas_final_modifier 02784_parallel_replicas_automatic_disabling +02581_share_big_sets_between_mutation_tasks_long +02581_share_big_sets_between_multiple_mutations_tasks_long diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index f77ef330ea2..9279b19b187 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -108,7 +108,7 @@ def get_run_command( env_str = " ".join(envs) volume_with_broken_test = ( - f"--volume={repo_tests_path}/broken_tests.txt:/broken_tests.txt" + f"--volume={repo_tests_path}/analyzer_tech_debt.txt:/analyzer_tech_debt.txt" if "analyzer" in check_name else "" ) From a487a1ab5acbe5fd2d5d1ee9872f895709408002 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 02:48:30 +0200 Subject: [PATCH 125/171] Fix bad test --- tests/queries/0_stateless/02479_mysql_connect_to_self.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql index c5c5dfb8c4a..cf2220073d3 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql @@ -2,6 +2,8 @@ SET send_logs_level = 'fatal'; -- failed connection tries are ok, if it succeeded after retry. +DROP TABLE IF EXISTS foo; + CREATE TABLE foo (key UInt32, a String, b Int64, c String) ENGINE = TinyLog; INSERT INTO foo VALUES (1, 'one', -1, 'een'), (2, 'two', -2, 'twee'), (3, 'three', -3, 'drie'), (4, 'four', -4, 'vier'), (5, 'five', -5, 'vijf'); @@ -40,3 +42,5 @@ SELECT * FROM mysql( SELECT '---'; SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connection_pool_size = 1, connect_timeout = 100, connection_wait_timeout = 100); SELECT count() FROM mysql('127.0.0.1:9004', currentDatabase(), foo, 'default', '', SETTINGS connection_pool_size = 0); -- { serverError BAD_ARGUMENTS } + +DROP TABLE foo; From 0e88aae9d7ab53d4267324e17f5a48fae00ae72c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 04:27:50 +0200 Subject: [PATCH 126/171] Remove ConsoleCertificateHandler --- .../Poco/Net/ConsoleCertificateHandler.h | 53 ------------------- .../include/Poco/Net/SSLManager.h | 6 +-- .../src/CertificateHandlerFactoryMgr.cpp | 2 - .../src/ConsoleCertificateHandler.cpp | 53 ------------------- base/poco/NetSSL_OpenSSL/src/SSLManager.cpp | 2 +- .../settings.md | 2 +- .../settings.md | 2 +- .../settings.md | 2 +- 8 files changed, 7 insertions(+), 115 deletions(-) delete mode 100644 base/poco/NetSSL_OpenSSL/include/Poco/Net/ConsoleCertificateHandler.h delete mode 100644 base/poco/NetSSL_OpenSSL/src/ConsoleCertificateHandler.cpp diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/ConsoleCertificateHandler.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/ConsoleCertificateHandler.h deleted file mode 100644 index 8e09b6f18ae..00000000000 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/ConsoleCertificateHandler.h +++ /dev/null @@ -1,53 +0,0 @@ -// -// ConsoleCertificateHandler.h -// -// Library: NetSSL_OpenSSL -// Package: SSLCore -// Module: ConsoleCertificateHandler -// -// Definition of the ConsoleCertificateHandler class. -// -// Copyright (c) 2006-2009, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#ifndef NetSSL_ConsoleCertificateHandler_INCLUDED -#define NetSSL_ConsoleCertificateHandler_INCLUDED - - -#include "Poco/Net/InvalidCertificateHandler.h" -#include "Poco/Net/NetSSL.h" - - -namespace Poco -{ -namespace Net -{ - - - class NetSSL_API ConsoleCertificateHandler : public InvalidCertificateHandler - /// A ConsoleCertificateHandler is invoked whenever an error occurs verifying the certificate. - /// - /// The certificate is printed to stdout and the user is asked via console if he wants to accept it. - { - public: - ConsoleCertificateHandler(bool handleErrorsOnServerSide); - /// Creates the ConsoleCertificateHandler. - - virtual ~ConsoleCertificateHandler(); - /// Destroys the ConsoleCertificateHandler. - - void onInvalidCertificate(const void * pSender, VerificationErrorArgs & errorCert); - /// Prints the certificate to stdout and waits for user input on the console - /// to decide if a certificate should be accepted/rejected. - }; - - -} -} // namespace Poco::Net - - -#endif // NetSSL_ConsoleCertificateHandler_INCLUDED diff --git a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h index a4fde26286e..21a1ed685e5 100644 --- a/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h +++ b/base/poco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h @@ -85,7 +85,7 @@ namespace Net /// /// /// - /// ConsoleCertificateHandler + /// RejectCertificateHandler /// /// true|false /// someString @@ -186,7 +186,7 @@ namespace Net /// /// Valid initialization code would be: /// SharedPtr pConsoleHandler = new KeyConsoleHandler; - /// SharedPtr pInvalidCertHandler = new ConsoleCertificateHandler; + /// SharedPtr pInvalidCertHandler = new RejectCertificateHandler; /// Context::Ptr pContext = new Context(Context::SERVER_USE, "any.pem", "any.pem", "rootcert.pem", Context::VERIFY_RELAXED, 9, false, "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); /// SSLManager::instance().initializeServer(pConsoleHandler, pInvalidCertHandler, pContext); @@ -203,7 +203,7 @@ namespace Net /// /// Valid initialization code would be: /// SharedPtr pConsoleHandler = new KeyConsoleHandler; - /// SharedPtr pInvalidCertHandler = new ConsoleCertificateHandler; + /// SharedPtr pInvalidCertHandler = new RejectCertificateHandler; /// Context::Ptr pContext = new Context(Context::CLIENT_USE, "", "", "rootcert.pem", Context::VERIFY_RELAXED, 9, false, "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH"); /// SSLManager::instance().initializeClient(pConsoleHandler, pInvalidCertHandler, pContext); diff --git a/base/poco/NetSSL_OpenSSL/src/CertificateHandlerFactoryMgr.cpp b/base/poco/NetSSL_OpenSSL/src/CertificateHandlerFactoryMgr.cpp index a89bbea11f2..f570e2d3599 100644 --- a/base/poco/NetSSL_OpenSSL/src/CertificateHandlerFactoryMgr.cpp +++ b/base/poco/NetSSL_OpenSSL/src/CertificateHandlerFactoryMgr.cpp @@ -13,7 +13,6 @@ #include "Poco/Net/CertificateHandlerFactoryMgr.h" -#include "Poco/Net/ConsoleCertificateHandler.h" #include "Poco/Net/AcceptCertificateHandler.h" #include "Poco/Net/RejectCertificateHandler.h" @@ -24,7 +23,6 @@ namespace Net { CertificateHandlerFactoryMgr::CertificateHandlerFactoryMgr() { - setFactory("ConsoleCertificateHandler", new CertificateHandlerFactoryImpl()); setFactory("AcceptCertificateHandler", new CertificateHandlerFactoryImpl()); setFactory("RejectCertificateHandler", new CertificateHandlerFactoryImpl()); } diff --git a/base/poco/NetSSL_OpenSSL/src/ConsoleCertificateHandler.cpp b/base/poco/NetSSL_OpenSSL/src/ConsoleCertificateHandler.cpp deleted file mode 100644 index db64752e70c..00000000000 --- a/base/poco/NetSSL_OpenSSL/src/ConsoleCertificateHandler.cpp +++ /dev/null @@ -1,53 +0,0 @@ -// -// ConsoleCertificateHandler.cpp -// -// Library: NetSSL_OpenSSL -// Package: SSLCore -// Module: ConsoleCertificateHandler -// -// Copyright (c) 2006-2009, Applied Informatics Software Engineering GmbH. -// and Contributors. -// -// SPDX-License-Identifier: BSL-1.0 -// - - -#include "Poco/Net/ConsoleCertificateHandler.h" -#include - - -namespace Poco { -namespace Net { - - -ConsoleCertificateHandler::ConsoleCertificateHandler(bool server): InvalidCertificateHandler(server) -{ -} - - -ConsoleCertificateHandler::~ConsoleCertificateHandler() -{ -} - - -void ConsoleCertificateHandler::onInvalidCertificate(const void*, VerificationErrorArgs& errorCert) -{ - const X509Certificate& aCert = errorCert.certificate(); - std::cout << "\n"; - std::cout << "WARNING: Certificate verification failed\n"; - std::cout << "----------------------------------------\n"; - std::cout << "Issuer Name: " << aCert.issuerName() << "\n"; - std::cout << "Subject Name: " << aCert.subjectName() << "\n\n"; - std::cout << "The certificate yielded the error: " << errorCert.errorMessage() << "\n\n"; - std::cout << "The error occurred in the certificate chain at position " << errorCert.errorDepth() << "\n"; - std::cout << "Accept the certificate (y,n)? "; - char c = 0; - std::cin >> c; - if (c == 'y' || c == 'Y') - errorCert.setIgnoreError(true); - else - errorCert.setIgnoreError(false); -} - - -} } // namespace Poco::Net diff --git a/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp b/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp index 82eed1a29eb..927602ca658 100644 --- a/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp +++ b/base/poco/NetSSL_OpenSSL/src/SSLManager.cpp @@ -46,7 +46,7 @@ const std::string SSLManager::CFG_PREFER_SERVER_CIPHERS("preferServerCiphers"); const std::string SSLManager::CFG_DELEGATE_HANDLER("privateKeyPassphraseHandler.name"); const std::string SSLManager::VAL_DELEGATE_HANDLER("KeyConsoleHandler"); const std::string SSLManager::CFG_CERTIFICATE_HANDLER("invalidCertificateHandler.name"); -const std::string SSLManager::VAL_CERTIFICATE_HANDLER("ConsoleCertificateHandler"); +const std::string SSLManager::VAL_CERTIFICATE_HANDLER("RejectCertificateHandler"); const std::string SSLManager::CFG_SERVER_PREFIX("openSSL.server."); const std::string SSLManager::CFG_CLIENT_PREFIX("openSSL.client."); const std::string SSLManager::CFG_CACHE_SESSIONS("cacheSessions"); diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 5643e273d39..40c1b8d64a1 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1602,7 +1602,7 @@ Keys for server/client settings: - requireTLSv1_2 (default: false) – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips (default: false) – Activates OpenSSL FIPS mode. Supported if the library’s OpenSSL version supports FIPS. - privateKeyPassphraseHandler (default: `KeyConsoleHandler`)– Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. -- invalidCertificateHandler (default: `ConsoleCertificateHandler`) – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` ConsoleCertificateHandler ` . +- invalidCertificateHandler (default: `RejectCertificateHandler`) – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` RejectCertificateHandler ` . - disableProtocols (default: "") – Protocols that are not allowed to use. - preferServerCiphers (default: false) – Preferred server ciphers on the client. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 33db6df0fdd..5430469ea18 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1067,7 +1067,7 @@ ClickHouse использует потоки из глобального пул - requireTLSv1_2 - Требование соединения TLSv1.2. Допустимые значения: `true`, `false`. - fips - Активация режима OpenSSL FIPS. Поддерживается, если версия OpenSSL, с которой собрана библиотека поддерживает fips. - privateKeyPassphraseHandler - Класс (подкласс PrivateKeyPassphraseHandler)запрашивающий кодовую фразу доступа к секретному ключу. Например, ``, `KeyFileHandler`, `test`, ``. -- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтверждения не валидных сертификатов. Например, ` ConsoleCertificateHandler `. +- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтверждения не валидных сертификатов. Например, ` RejectCertificateHandler `. - disableProtocols - Запрещенные к использованию протоколы. - preferServerCiphers - Предпочтение серверных шифров на клиенте. diff --git a/docs/zh/operations/server-configuration-parameters/settings.md b/docs/zh/operations/server-configuration-parameters/settings.md index 52142eda2e8..f6106d8734e 100644 --- a/docs/zh/operations/server-configuration-parameters/settings.md +++ b/docs/zh/operations/server-configuration-parameters/settings.md @@ -466,7 +466,7 @@ SSL客户端/服务器配置。 - requireTLSv1_2 – Require a TLSv1.2 connection. Acceptable values: `true`, `false`. - fips – Activates OpenSSL FIPS mode. Supported if the library’s OpenSSL version supports FIPS. - privateKeyPassphraseHandler – Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: ``, `KeyFileHandler`, `test`, ``. -- invalidCertificateHandler – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` ConsoleCertificateHandler ` . +- invalidCertificateHandler – Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: ` RejectCertificateHandler ` . - disableProtocols – Protocols that are not allowed to use. - preferServerCiphers – Preferred server ciphers on the client. From 88048153d7af70dcfc27dc1c9bd480d9cad99d53 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 04:37:14 +0200 Subject: [PATCH 127/171] Fix 00899_long_attach_memory_limit --- tests/queries/0_stateless/00899_long_attach_memory_limit.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00899_long_attach_memory_limit.sql b/tests/queries/0_stateless/00899_long_attach_memory_limit.sql index aa507cda2ac..d4aa2a0eb7b 100644 --- a/tests/queries/0_stateless/00899_long_attach_memory_limit.sql +++ b/tests/queries/0_stateless/00899_long_attach_memory_limit.sql @@ -1,4 +1,5 @@ --- Tags: long, no-debug, no-parallel, no-fasttest +-- Tags: long, no-debug, no-parallel, no-fasttest, no-msan, no-tsan +-- This test is slow under MSan or TSan. DROP TABLE IF EXISTS index_memory; CREATE TABLE index_memory (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1; From 4cef0342979ced426f4823016ebc65f42067aab7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:27:45 +0200 Subject: [PATCH 128/171] Fix 02481_parquet_list_monotonically_increasing_offsets.sh --- .../02481_parquet_list_monotonically_increasing_offsets.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh b/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh index 47245eeb940..55e6ac2f758 100755 --- a/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh +++ b/tests/queries/0_stateless/02481_parquet_list_monotonically_increasing_offsets.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash -# Tags: no-ubsan, no-fasttest +# Tags: no-ubsan, no-fasttest, no-tsan +# It is too slow under TSan CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -13,4 +14,4 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_load (list Array(Int64), json cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "INSERT INTO parquet_load FORMAT Parquet" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load" | md5sum ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM parquet_load" -${CLICKHOUSE_CLIENT} --query="drop table parquet_load" \ No newline at end of file +${CLICKHOUSE_CLIENT} --query="drop table parquet_load" From e3d999b023cad921dd740354125a8cbac9c30da5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:43:27 +0200 Subject: [PATCH 129/171] Fix test 02497_trace_events_stress_long --- .../02497_trace_events_stress_long.sh | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02497_trace_events_stress_long.sh b/tests/queries/0_stateless/02497_trace_events_stress_long.sh index 7dc72807c5a..f1dbb9c0399 100755 --- a/tests/queries/0_stateless/02497_trace_events_stress_long.sh +++ b/tests/queries/0_stateless/02497_trace_events_stress_long.sh @@ -44,4 +44,18 @@ timeout $TIMEOUT bash -c thread2 >/dev/null & wait $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%' SYNC" >/dev/null -$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" + +# After this moment, the server can still run another query. +# For example, the 'timeout' command killed all threads of thread1, +# and the 'timeout' itself has finished, and we have successfully 'wait'-ed for it, +# but just before that, one of the threads successfully sent a query to the server, +# but the server didn't start to run this query yet, +# and even when the KILL QUERY was run, the query from the thread didn't start, +# but only started after the KILL QUERY has been already processed. + +# That's why we have to run the next command in a loop. + +for _ in {1..10} +do + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' && break +done From 21c9feeeb3af922fb621688d8bfc7459d9a50f1a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:46:10 +0200 Subject: [PATCH 130/171] Fix test --- .../02497_trace_events_stress_long.sh | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02497_trace_events_stress_long.sh b/tests/queries/0_stateless/02497_trace_events_stress_long.sh index f1dbb9c0399..3ec729079b8 100755 --- a/tests/queries/0_stateless/02497_trace_events_stress_long.sh +++ b/tests/queries/0_stateless/02497_trace_events_stress_long.sh @@ -43,19 +43,21 @@ timeout $TIMEOUT bash -c thread2 >/dev/null & wait -$CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%' SYNC" >/dev/null - -# After this moment, the server can still run another query. -# For example, the 'timeout' command killed all threads of thread1, -# and the 'timeout' itself has finished, and we have successfully 'wait'-ed for it, -# but just before that, one of the threads successfully sent a query to the server, -# but the server didn't start to run this query yet, -# and even when the KILL QUERY was run, the query from the thread didn't start, -# but only started after the KILL QUERY has been already processed. - -# That's why we have to run the next command in a loop. - for _ in {1..10} do + $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%' SYNC" >/dev/null + + # After this moment, the server can still run another query. + # For example, the 'timeout' command killed all threads of thread1, + # and the 'timeout' itself has finished, and we have successfully 'wait'-ed for it, + # but just before that, one of the threads successfully sent a query to the server, + # but the server didn't start to run this query yet, + # and even when the KILL QUERY was run, the query from the thread didn't start, + # but only started after the KILL QUERY has been already processed. + + # That's why we have to run this in a loop. + $CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id LIKE '02497_$CLICKHOUSE_DATABASE%'" | rg '^0$' && break + + sleep 1 done From 8e6f6655853cfd1106f96c75fddb4cd57f39f8a0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Jun 2023 05:57:38 +0200 Subject: [PATCH 131/171] Fix build --- src/Functions/IFunction.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index 1e4f8bf1102..c5b9a78015d 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -11,6 +11,11 @@ #include +#if USE_EMBEDDED_COMPILER +# include +#endif + + /// This file contains user interface for functions. namespace llvm From e92035072b7b3367da12089b28041893eb90e636 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 26 Jun 2023 13:44:09 +0300 Subject: [PATCH 132/171] Update MergeTreeTransaction.cpp --- src/Interpreters/MergeTreeTransaction.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/MergeTreeTransaction.cpp b/src/Interpreters/MergeTreeTransaction.cpp index 1358e3ed3c2..6b8e09a64f5 100644 --- a/src/Interpreters/MergeTreeTransaction.cpp +++ b/src/Interpreters/MergeTreeTransaction.cpp @@ -326,6 +326,8 @@ void MergeTreeTransaction::afterFinalize() is_read_only = storages.empty(); /// Release shared pointers just in case + creating_parts.clear(); + removing_parts.clear(); storages.clear(); mutations.clear(); finalized = true; From eb649873b379a29fec5584205558f70641f16bba Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 26 Jun 2023 13:48:21 +0300 Subject: [PATCH 133/171] Unify merge predicate (#51344) * unify merge predicate * Update BaseDaemon.cpp --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 234 +++++++++--------- .../MergeTree/ReplicatedMergeTreeQueue.h | 95 +++---- 2 files changed, 175 insertions(+), 154 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 03ded2ef260..3ba3048b812 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1530,7 +1530,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( Int64 ReplicatedMergeTreeQueue::getCurrentMutationVersion( - const String & partition_id, Int64 data_version, std::lock_guard & /* state_lock */) const + const String & partition_id, Int64 data_version) const { auto in_partition = mutations_by_partition.find(partition_id); if (in_partition == mutations_by_partition.end()) @@ -2108,24 +2108,19 @@ ReplicatedMergeTreeQueue::QueueLocks ReplicatedMergeTreeQueue::lockQueue() } LocalMergePredicate::LocalMergePredicate(ReplicatedMergeTreeQueue & queue_) - : queue(queue_) { + /// Use only information that can be quickly accessed locally without querying ZooKeeper + virtual_parts_ = &queue_.virtual_parts; + mutations_state_ = &queue_; + virtual_parts_mutex = &queue_.state_mutex; } -ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( - ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, std::optional && partition_ids_hint_) - : nested_pred(queue_) - , queue(queue_) - , partition_ids_hint(std::move(partition_ids_hint_)) - , prev_virtual_parts(queue.format_version) -{ - { - std::lock_guard lock(queue.state_mutex); - prev_virtual_parts = queue.virtual_parts; - } - /// Load current quorum status. - auto quorum_status_future = zookeeper->asyncTryGet(fs::path(queue.zookeeper_path) / "quorum" / "status"); +template +CommittingBlocks BaseMergePredicate::getCommittingBlocks( + zkutil::ZooKeeperPtr & zookeeper, const std::string & zookeeper_path, Poco::Logger * log_) +{ + CommittingBlocks committing_blocks; /// Load current inserts /// Hint avoids listing partitions that we don't really need. @@ -2133,14 +2128,14 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( /// so without hint it can do a few thousands requests (if not using MultiRead). Strings partitions; if (!partition_ids_hint) - partitions = zookeeper->getChildren(fs::path(queue.zookeeper_path) / "block_numbers"); + partitions = zookeeper->getChildren(fs::path(zookeeper_path) / "block_numbers"); else std::copy(partition_ids_hint->begin(), partition_ids_hint->end(), std::back_inserter(partitions)); std::vector paths; paths.reserve(partitions.size()); for (const String & partition : partitions) - paths.push_back(fs::path(queue.zookeeper_path) / "block_numbers" / partition); + paths.push_back(fs::path(zookeeper_path) / "block_numbers" / partition); auto locks_children = zookeeper->tryGetChildren(paths); @@ -2153,22 +2148,40 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( if (response.error != Coordination::Error::ZOK) { /// Probably a wrong hint was provided (it's ok if a user passed non-existing partition to OPTIMIZE) - LOG_WARNING(queue.log, "Partition id '{}' was provided as a hint, but there's not such partition in ZooKeeper", partitions[i]); + LOG_WARNING(log_, "Partition id '{}' was provided as a hint, but there's not such partition in ZooKeeper", partitions[i]); partition_ids_hint->erase(partitions[i]); continue; } - Strings partition_block_numbers = response.names; + Strings partition_block_numbers = locks_children[i].names; for (const String & entry : partition_block_numbers) { if (!startsWith(entry, "block-")) continue; + Int64 block_number = parse(entry.substr(strlen("block-"))); - String zk_path = fs::path(queue.zookeeper_path) / "block_numbers" / partitions[i] / entry; committing_blocks[partitions[i]].insert(block_number); } } + return committing_blocks; +} + +ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( + ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, std::optional && partition_ids_hint_) + : BaseMergePredicate(std::move(partition_ids_hint_)) + , queue(queue_) +{ + { + std::lock_guard lock(queue.state_mutex); + prev_virtual_parts = std::make_shared(queue.virtual_parts); + } + + /// Load current quorum status. + auto quorum_status_future = zookeeper->asyncTryGet(fs::path(queue.zookeeper_path) / "quorum" / "status"); + + committing_blocks = std::make_shared(getCommittingBlocks(zookeeper, queue.zookeeper_path, queue.log)); + merges_version = queue_.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::MERGE_PREDICATE); { @@ -2179,7 +2192,8 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( /// /// If pinned parts are fetched after logs are pulled then we can safely say that it contains all locks up to `merges_version`. String s = zookeeper->get(queue.zookeeper_path + "/pinned_part_uuids"); - pinned_part_uuids.fromString(s); + pinned_part_uuids = std::make_shared(); + pinned_part_uuids->fromString(s); } Coordination::GetResponse quorum_status_response = quorum_status_future.get(); @@ -2187,13 +2201,21 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( { ReplicatedMergeTreeQuorumEntry quorum_status; quorum_status.fromString(quorum_status_response.data); - inprogress_quorum_part = quorum_status.part_name; + inprogress_quorum_part = std::make_shared(quorum_status.part_name); } - else - inprogress_quorum_part.clear(); + + /// Use all information about parts + prev_virtual_parts_ = prev_virtual_parts.get(); + virtual_parts_ = &queue.virtual_parts; + committing_blocks_ = committing_blocks.get(); + pinned_part_uuids_ = pinned_part_uuids.get(); + inprogress_quorum_part_ = inprogress_quorum_part.get(); + mutations_state_ = &queue; + virtual_parts_mutex = &queue.state_mutex; } -bool LocalMergePredicate::operator()( +template +bool BaseMergePredicate::operator()( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, const MergeTreeTransaction *, @@ -2205,20 +2227,8 @@ bool LocalMergePredicate::operator()( return canMergeSinglePart(right, out_reason); } -bool ReplicatedMergeTreeMergePredicate::operator()( - const MergeTreeData::DataPartPtr & left, - const MergeTreeData::DataPartPtr & right, - const MergeTreeTransaction *, - String * out_reason) const -{ - if (left) - return canMergeTwoParts(left, right, out_reason); - else - return canMergeSinglePart(right, out_reason); -} - - -bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( +template +bool BaseMergePredicate::canMergeTwoParts( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, String * out_reason) const @@ -2263,21 +2273,21 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( for (const MergeTreeData::DataPartPtr & part : {left, right}) { - if (pinned_part_uuids.part_uuids.contains(part->uuid)) + if (pinned_part_uuids_ && pinned_part_uuids_->part_uuids.contains(part->uuid)) { if (out_reason) *out_reason = "Part " + part->name + " has uuid " + toString(part->uuid) + " which is currently pinned"; return false; } - if (part->name == inprogress_quorum_part) + if (inprogress_quorum_part_ && part->name == *inprogress_quorum_part_) { if (out_reason) *out_reason = "Quorum insert for part " + part->name + " is currently in progress"; return false; } - if (prev_virtual_parts.getContainingPart(part->info).empty()) + if (prev_virtual_parts_ && prev_virtual_parts_->getContainingPart(part->info).empty()) { if (out_reason) *out_reason = "Entry for part " + part->name + " hasn't been read from the replication log yet"; @@ -2290,7 +2300,7 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( if (left_max_block > right_min_block) std::swap(left_max_block, right_min_block); - if (left_max_block + 1 < right_min_block) + if (committing_blocks_ && left_max_block + 1 < right_min_block) { if (partition_ids_hint && !partition_ids_hint->contains(left->info.partition_id)) { @@ -2299,8 +2309,8 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( return false; } - auto committing_blocks_in_partition = committing_blocks.find(left->info.partition_id); - if (committing_blocks_in_partition != committing_blocks.end()) + auto committing_blocks_in_partition = committing_blocks_->find(left->info.partition_id); + if (committing_blocks_in_partition != committing_blocks_->end()) { const std::set & block_numbers = committing_blocks_in_partition->second; @@ -2316,109 +2326,107 @@ bool ReplicatedMergeTreeMergePredicate::canMergeTwoParts( } } - return nested_pred.canMergeTwoParts(left, right, out_reason); -} + std::unique_lock lock; + if (virtual_parts_mutex) + lock = std::unique_lock(*virtual_parts_mutex); -bool LocalMergePredicate::canMergeTwoParts( - const MergeTreeData::DataPartPtr & left, - const MergeTreeData::DataPartPtr & right, - String * out_reason) const -{ - Int64 left_max_block = left->info.max_block; - Int64 right_min_block = right->info.min_block; - - std::lock_guard lock(queue.state_mutex); - - for (const MergeTreeData::DataPartPtr & part : {left, right}) + if (virtual_parts_) { - /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer - /// and it is guaranteed that it will contain all merges assigned before this object is constructed. - String containing_part = queue.virtual_parts.getContainingPart(part->info); - if (containing_part != part->name) + for (const MergeTreeData::DataPartPtr & part : {left, right}) { - if (out_reason) - *out_reason = "Part " + part->name + " has already been assigned a merge into " + containing_part; - return false; + /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer + /// and it is guaranteed that it will contain all merges assigned before this object is constructed. + String containing_part = virtual_parts_->getContainingPart(part->info); + if (containing_part != part->name) + { + if (out_reason) + *out_reason = "Part " + part->name + " has already been assigned a merge into " + containing_part; + return false; + } + } + + if (left_max_block + 1 < right_min_block) + { + /// Fake part which will appear as merge result + MergeTreePartInfo gap_part_info( + left->info.partition_id, left_max_block + 1, right_min_block - 1, + MergeTreePartInfo::MAX_LEVEL, MergeTreePartInfo::MAX_BLOCK_NUMBER); + + /// We don't select parts if any smaller part covered by our merge must exist after + /// processing replication log up to log_pointer. + Strings covered = virtual_parts_->getPartsCoveredBy(gap_part_info); + if (!covered.empty()) + { + if (out_reason) + *out_reason = "There are " + toString(covered.size()) + " parts (from " + covered.front() + + " to " + covered.back() + ") that are still not present or being processed by " + + " other background process on this replica between " + left->name + " and " + right->name; + return false; + } } } - if (left_max_block + 1 < right_min_block) + if (mutations_state_) { - /// Fake part which will appear as merge result - MergeTreePartInfo gap_part_info( - left->info.partition_id, left_max_block + 1, right_min_block - 1, - MergeTreePartInfo::MAX_LEVEL, MergeTreePartInfo::MAX_BLOCK_NUMBER); + Int64 left_mutation_ver = mutations_state_->getCurrentMutationVersion( + left->info.partition_id, left->info.getDataVersion()); - /// We don't select parts if any smaller part covered by our merge must exist after - /// processing replication log up to log_pointer. - Strings covered = queue.virtual_parts.getPartsCoveredBy(gap_part_info); - if (!covered.empty()) + Int64 right_mutation_ver = mutations_state_->getCurrentMutationVersion( + left->info.partition_id, right->info.getDataVersion()); + + if (left_mutation_ver != right_mutation_ver) { if (out_reason) - *out_reason = "There are " + toString(covered.size()) + " parts (from " + covered.front() - + " to " + covered.back() + ") that are still not present or being processed by " - + " other background process on this replica between " + left->name + " and " + right->name; + *out_reason = "Current mutation versions of parts " + left->name + " and " + right->name + " differ: " + + toString(left_mutation_ver) + " and " + toString(right_mutation_ver) + " respectively"; return false; } } - Int64 left_mutation_ver = queue.getCurrentMutationVersion( - left->info.partition_id, left->info.getDataVersion(), lock); - - Int64 right_mutation_ver = queue.getCurrentMutationVersion( - left->info.partition_id, right->info.getDataVersion(), lock); - - if (left_mutation_ver != right_mutation_ver) - { - if (out_reason) - *out_reason = "Current mutation versions of parts " + left->name + " and " + right->name + " differ: " - + toString(left_mutation_ver) + " and " + toString(right_mutation_ver) + " respectively"; - return false; - } - return MergeTreeData::partsContainSameProjections(left, right); } -bool ReplicatedMergeTreeMergePredicate::canMergeSinglePart( +template +bool BaseMergePredicate::canMergeSinglePart( const MergeTreeData::DataPartPtr & part, String * out_reason) const { - if (pinned_part_uuids.part_uuids.contains(part->uuid)) + if (pinned_part_uuids_ && pinned_part_uuids_->part_uuids.contains(part->uuid)) { if (out_reason) *out_reason = fmt::format("Part {} has uuid {} which is currently pinned", part->name, part->uuid); return false; } - if (part->name == inprogress_quorum_part) + if (inprogress_quorum_part_ && part->name == *inprogress_quorum_part_) { if (out_reason) *out_reason = fmt::format("Quorum insert for part {} is currently in progress", part->name); return false; } - if (prev_virtual_parts.getContainingPart(part->info).empty()) + if (prev_virtual_parts_ && prev_virtual_parts_->getContainingPart(part->info).empty()) { if (out_reason) *out_reason = fmt::format("Entry for part {} hasn't been read from the replication log yet", part->name); return false; } - return nested_pred.canMergeSinglePart(part, out_reason); -} + std::unique_lock lock; + if (virtual_parts_mutex) + lock = std::unique_lock(*virtual_parts_mutex); -bool LocalMergePredicate::canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const -{ - std::lock_guard lock(queue.state_mutex); - - /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer - /// and it is guaranteed that it will contain all merges assigned before this object is constructed. - String containing_part = queue.virtual_parts.getContainingPart(part->info); - if (containing_part != part->name) + if (virtual_parts_) { - if (out_reason) - *out_reason = fmt::format("Part {} has already been assigned a merge into {}", part->name, containing_part); - return false; + /// We look for containing parts in queue.virtual_parts (and not in prev_virtual_parts) because queue.virtual_parts is newer + /// and it is guaranteed that it will contain all merges assigned before this object is constructed. + String containing_part = virtual_parts_->getContainingPart(part->info); + if (containing_part != part->name) + { + if (out_reason) + *out_reason = fmt::format("Part {} has already been assigned a merge into {}", part->name, containing_part); + return false; + } } return true; @@ -2459,7 +2467,7 @@ std::optional> ReplicatedMergeTreeMergePredicate::getDesir /// We cannot mutate part if it's being inserted with quorum and it's not /// already reached. - if (part->name == inprogress_quorum_part) + if (inprogress_quorum_part && part->name == *inprogress_quorum_part) return {}; std::lock_guard lock(queue.state_mutex); @@ -2474,7 +2482,7 @@ std::optional> ReplicatedMergeTreeMergePredicate::getDesir UInt64 mutations_limit = queue.storage.getSettings()->replicated_max_mutations_in_one_entry; UInt64 mutations_count = 0; - Int64 current_version = queue.getCurrentMutationVersion(part->info.partition_id, part->info.getDataVersion(), lock); + Int64 current_version = queue.getCurrentMutationVersion(part->info.partition_id, part->info.getDataVersion()); Int64 max_version = in_partition->second.begin()->first; int alter_version = -1; @@ -2548,8 +2556,8 @@ bool ReplicatedMergeTreeMergePredicate::isMutationFinished(const std::string & z if (partition_ids_hint && !partition_ids_hint->contains(partition_id)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Partition id {} was not provided as hint, it's a bug", partition_id); - auto partition_it = committing_blocks.find(partition_id); - if (partition_it != committing_blocks.end()) + auto partition_it = committing_blocks->find(partition_id); + if (partition_it != committing_blocks->end()) { size_t blocks_count = std::distance( partition_it->second.begin(), partition_it->second.lower_bound(block_num)); @@ -2671,4 +2679,6 @@ void ReplicatedMergeTreeQueue::removeCurrentPartsFromMutations() removeCoveredPartsFromMutations(part_name, /*remove_part = */ false, /*remove_covered_parts = */ true); } +template class BaseMergePredicate; + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 79572e13963..f205526a660 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -34,6 +34,7 @@ private: friend class CurrentlyExecuting; friend class LocalMergePredicate; friend class ReplicatedMergeTreeMergePredicate; + template friend class BaseMergePredicate; friend class MergeFromLogEntryTask; friend class ReplicatedMergeMutateTaskBase; @@ -212,7 +213,7 @@ private: /// with getDataVersion() == data_version. (Either this mutation was already applied or the part /// was created after the mutation). /// If there is no such mutation or it has already been executed and deleted, return 0. - Int64 getCurrentMutationVersion(const String & partition_id, Int64 data_version, std::lock_guard & /* state_lock */) const; + Int64 getCurrentMutationVersion(const String & partition_id, Int64 data_version) const; /** Check that part isn't in currently generating parts and isn't covered by them. * Should be called under state_mutex. @@ -491,33 +492,14 @@ public: void createLogEntriesToFetchBrokenParts(); }; -/// Lightweight version of ReplicatedMergeTreeMergePredicate that do not make any ZooKeeper requests, -/// but may return false-positive results. Checks only a subset of required conditions. -class LocalMergePredicate +using CommittingBlocks = std::unordered_map>; + +template +class BaseMergePredicate { public: - LocalMergePredicate(ReplicatedMergeTreeQueue & queue_); - - bool operator()(const MergeTreeData::DataPartPtr & left, - const MergeTreeData::DataPartPtr & right, - const MergeTreeTransaction * txn, - String * out_reason = nullptr) const; - - bool canMergeTwoParts(const MergeTreeData::DataPartPtr & left, - const MergeTreeData::DataPartPtr & right, - String * out_reason = nullptr) const; - - bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const; - -private: - const ReplicatedMergeTreeQueue & queue; -}; - -class ReplicatedMergeTreeMergePredicate -{ -public: - ReplicatedMergeTreeMergePredicate(ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, - std::optional && partition_ids_hint_); + BaseMergePredicate() = default; + BaseMergePredicate(std::optional && partition_ids_hint_) : partition_ids_hint(std::move(partition_ids_hint_)) {} /// Depending on the existence of left part checks a merge predicate for two parts or for single part. bool operator()(const MergeTreeData::DataPartPtr & left, @@ -537,6 +519,46 @@ public: /// This predicate is checked for the first part of each range. bool canMergeSinglePart(const MergeTreeData::DataPartPtr & part, String * out_reason) const; + CommittingBlocks getCommittingBlocks(zkutil::ZooKeeperPtr & zookeeper, const std::string & zookeeper_path, Poco::Logger * log_); + +protected: + /// A list of partitions that can be used in the merge predicate + std::optional partition_ids_hint; + + /// A snapshot of active parts that would appear if the replica executes all log entries in its queue. + const VirtualPartsT * prev_virtual_parts_ = nullptr; + const VirtualPartsT * virtual_parts_ = nullptr; + + /// partition ID -> block numbers of the inserts and mutations that are about to commit + /// (loaded at some later time than prev_virtual_parts). + const CommittingBlocks * committing_blocks_ = nullptr; + + /// List of UUIDs for parts that have their identity "pinned". + const PinnedPartUUIDs * pinned_part_uuids_ = nullptr; + + /// Quorum state taken at some later time than prev_virtual_parts. + const String * inprogress_quorum_part_ = nullptr; + + /// An object that provides current mutation version for a part + const MutationsStateT * mutations_state_ = nullptr; + + std::mutex * virtual_parts_mutex = nullptr; +}; + +/// Lightweight version of ReplicatedMergeTreeMergePredicate that do not make any ZooKeeper requests, +/// but may return false-positive results. Checks only a subset of required conditions. +class LocalMergePredicate : public BaseMergePredicate +{ +public: + LocalMergePredicate(ReplicatedMergeTreeQueue & queue_); +}; + +class ReplicatedMergeTreeMergePredicate : public BaseMergePredicate +{ +public: + ReplicatedMergeTreeMergePredicate(ReplicatedMergeTreeQueue & queue_, zkutil::ZooKeeperPtr & zookeeper, + std::optional && partition_ids_hint_); + /// Returns true if part is needed for some REPLACE_RANGE entry. /// We should not drop part in this case, because replication queue may stuck without that part. bool partParticipatesInReplaceRange(const MergeTreeData::DataPartPtr & part, String * out_reason) const; @@ -561,28 +583,17 @@ public: String getCoveringVirtualPart(const String & part_name) const; private: - LocalMergePredicate nested_pred; - const ReplicatedMergeTreeQueue & queue; - std::optional partition_ids_hint; - - /// A snapshot of active parts that would appear if the replica executes all log entries in its queue. - ActiveDataPartSet prev_virtual_parts; - /// partition ID -> block numbers of the inserts and mutations that are about to commit - /// (loaded at some later time than prev_virtual_parts). - std::unordered_map> committing_blocks; - - /// List of UUIDs for parts that have their identity "pinned". - PinnedPartUUIDs pinned_part_uuids; - - /// Quorum state taken at some later time than prev_virtual_parts. - String inprogress_quorum_part; + /// We copy a merge predicate when we cast it to AllowedMergingPredicate, let's keep the pointers valid + std::shared_ptr prev_virtual_parts; + std::shared_ptr committing_blocks; + std::shared_ptr pinned_part_uuids; + std::shared_ptr inprogress_quorum_part; int32_t merges_version = -1; }; - /** Convert a number to a string in the format of the suffixes of auto-incremental nodes in ZooKeeper. * Negative numbers are also supported - for them the name of the node looks somewhat silly * and does not match any auto-incremented node in ZK. From c64f929b9cb57ca171dbe28e82b60a956004daf8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 26 Jun 2023 12:10:58 +0200 Subject: [PATCH 134/171] Fix broken labeling for `manual approve` --- tests/ci/workflow_approve_rerun_lambda/app.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 3db62430d85..1a3874ad01a 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -224,8 +224,8 @@ def approve_run(workflow_description: WorkflowDescription, token: str) -> None: def label_manual_approve(pull_request, token): - url = f"{pull_request['url']}/labels" - data = {"labels": "manual approve"} + url = f"{pull_request['issue_url']}/labels" + data = {"labels": ["manual approve"]} _exec_post_with_retry(url, token, data) From 065b87aa30d6e6a3f7e64ef3fdfabdc79039a91d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 26 Jun 2023 12:21:20 +0200 Subject: [PATCH 135/171] Improve logging a little bit, avoid double labeling --- tests/ci/workflow_approve_rerun_lambda/app.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 1a3874ad01a..5e2331ece3c 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -376,11 +376,10 @@ def main(event): changed_files = get_changed_files_for_pull_request(pull_request, token) print(f"Totally have {len(changed_files)} changed files in PR:", changed_files) if check_suspicious_changed_files(changed_files): - print( - f"Pull Request {pull_request['number']} has suspicious changes, " - "label it for manuall approve" - ) - label_manual_approve(pull_request, token) + print(f"Pull Request {pull_request['number']} has suspicious changes") + if "manual approve" not in labels: + print("Label the PR as needed for manuall approve") + label_manual_approve(pull_request, token) else: print(f"Pull Request {pull_request['number']} has no suspicious changes") approve_run(workflow_description, token) From 9af56624f523b63a078815fc5e31bc16f49821c9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 26 Jun 2023 13:30:51 +0200 Subject: [PATCH 136/171] Fix flaky test test_skip_empty_files --- tests/integration/test_storage_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index a7293337a9e..6c251d2f84e 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1776,7 +1776,7 @@ def test_skip_empty_files(started_cluster): assert int(res) == 0 res = instance.query( - f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{11|1|22}}.parquet') settings engine_url_skip_empty_files=1" + f"select * from url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/skip_empty_files{{11|1|22}}.parquet', auto, 'number UInt64') settings engine_url_skip_empty_files=1" ) assert len(res.strip()) == 0 From c4d190617c72436bfeae8563c636ed8e925e1849 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Mon, 26 Jun 2023 14:46:51 +0200 Subject: [PATCH 137/171] fix flacky test test_profile_events_s3 --- tests/integration/test_profile_events_s3/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index 10c9385f865..de18142d4f6 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -139,7 +139,7 @@ def test_profile_events(cluster): ) stat1 = get_query_stat(instance, query1) for metric in stat1: - assert stat1[metric] == metrics1[metric] - metrics0[metric] + assert stat1[metric] == metrics1.get(metric, 0) - metrics0.get(metric, 0) assert ( metrics1["WriteBufferFromS3Bytes"] - metrics0["WriteBufferFromS3Bytes"] == size1 ) @@ -163,7 +163,7 @@ def test_profile_events(cluster): stat2 = get_query_stat(instance, query2) for metric in stat2: - assert stat2[metric] == metrics2[metric] - metrics1[metric] + assert stat2[metric] == metrics2.get(metric, 0)- metrics1.get(metric, 0) assert ( metrics2["WriteBufferFromS3Bytes"] - metrics1["WriteBufferFromS3Bytes"] @@ -189,4 +189,4 @@ def test_profile_events(cluster): # With async reads profile events are not updated fully because reads are done in a separate thread. # for metric in stat3: # print(metric) - # assert stat3[metric] == metrics3[metric] - metrics2[metric] + # assert stat3[metric] == metrics3.get(metric, 0) - metrics2.get(metric, 0) From 4fabce9cc85cf649af55c0bb5341dad15aa91231 Mon Sep 17 00:00:00 2001 From: Tyler Hannan Date: Mon, 26 Jun 2023 15:10:56 +0200 Subject: [PATCH 138/171] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 87fb4920222..eac036c2d9b 100644 --- a/README.md +++ b/README.md @@ -27,6 +27,7 @@ curl https://clickhouse.com/ | sh * [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/293913596) - Jul 18 * [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/293913441) - Jul 19 * [**ClickHouse Meetup in Toronto**](https://www.meetup.com/clickhouse-toronto-user-group/events/294183127) - Jul 20 +* [**ClickHouse Meetup in Singapore**](https://www.meetup.com/clickhouse-singapore-meetup-group/events/294428050/) - Jul 27 Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler clickhouse com. From 237805705c0239d3b252b3fa3b59f0214452ccd0 Mon Sep 17 00:00:00 2001 From: Sema Checherinda <104093494+CheSema@users.noreply.github.com> Date: Mon, 26 Jun 2023 15:44:30 +0200 Subject: [PATCH 139/171] Update test.py --- tests/integration/test_profile_events_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index de18142d4f6..a8e037d667f 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -163,7 +163,7 @@ def test_profile_events(cluster): stat2 = get_query_stat(instance, query2) for metric in stat2: - assert stat2[metric] == metrics2.get(metric, 0)- metrics1.get(metric, 0) + assert stat2[metric] == metrics2.get(metric, 0) - metrics1.get(metric, 0) assert ( metrics2["WriteBufferFromS3Bytes"] - metrics1["WriteBufferFromS3Bytes"] From 7cadfeac29c68dc9ac749cd48d115da02ae48f62 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1nos=20Benjamin=20Antal?= Date: Mon, 26 Jun 2023 14:55:03 +0000 Subject: [PATCH 140/171] Add retries to flaky tests --- tests/integration/test_tlsv1_3/test.py | 67 ++++++++++++++++++-------- 1 file changed, 47 insertions(+), 20 deletions(-) diff --git a/tests/integration/test_tlsv1_3/test.py b/tests/integration/test_tlsv1_3/test.py index e25216c67df..a29dcb5fc1c 100644 --- a/tests/integration/test_tlsv1_3/test.py +++ b/tests/integration/test_tlsv1_3/test.py @@ -4,6 +4,7 @@ from helpers.ssl_context import WrapSSLContextWithSNI import urllib.request, urllib.parse import ssl import os.path +import logging # The test cluster is configured with certificate for that host name, see 'server-ext.cnf'. @@ -11,6 +12,7 @@ import os.path SSL_HOST = "integration-tests.clickhouse.com" HTTPS_PORT = 8443 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +MAX_RETRY = 5 cluster = ClickHouseCluster(__file__) instance = cluster.add_instance( @@ -88,10 +90,18 @@ def test_https_wrong_cert(): execute_query_https("SELECT currentUser()", user="john", cert_name="client2") assert "HTTP Error 403" in str(err.value) + count = 0 # Wrong certificate: self-signed certificate. - with pytest.raises(Exception) as err: - execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") - assert "unknown ca" in str(err.value) + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https("SELECT currentUser()", user="john", cert_name="wrong") + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning(f"Failed attempt with wrong cert, err: {err_str}") + continue + assert "unknown ca" in err_str + break # No certificate. with pytest.raises(Exception) as err: @@ -181,24 +191,41 @@ def test_https_non_ssl_auth(): == "jane\n" ) + count = 0 # However if we send a certificate it must not be wrong. - with pytest.raises(Exception) as err: - execute_query_https( - "SELECT currentUser()", - user="peter", - enable_ssl_auth=False, - cert_name="wrong", - ) - assert "unknown ca" in str(err.value) - with pytest.raises(Exception) as err: - execute_query_https( - "SELECT currentUser()", - user="jane", - enable_ssl_auth=False, - password="qwe123", - cert_name="wrong", - ) - assert "unknown ca" in str(err.value) + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https( + "SELECT currentUser()", + user="peter", + enable_ssl_auth=False, + cert_name="wrong", + ) + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning(f"Failed attempt with wrong cert, user: peter, err: {err_str}") + continue + assert "unknown ca" in err_str + break + + count = 0 + while count <= MAX_RETRY: + with pytest.raises(Exception) as err: + execute_query_https( + "SELECT currentUser()", + user="jane", + enable_ssl_auth=False, + password="qwe123", + cert_name="wrong", + ) + err_str = str(err.value) + if count < MAX_RETRY and "Broken pipe" in err_str: + count = count + 1 + logging.warning(f"Failed attempt with wrong cert, user: jane, err: {err_str}") + continue + assert "unknown ca" in err_str + break def test_create_user(): From 45c263cbb551c27e303f6a050df558ec63a3b463 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 15:00:46 +0000 Subject: [PATCH 141/171] Replace try/catch logic in hasTokenOrNull() by sth more lightweight --- src/Common/StringSearcher.h | 16 ++++- src/Common/Volnitsky.h | 3 +- src/Functions/HasTokenImpl.h | 74 +++++++++++------------ src/Functions/hasToken.cpp | 12 ++-- src/Functions/hasTokenCaseInsensitive.cpp | 7 +-- 5 files changed, 59 insertions(+), 53 deletions(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index ae440f9151b..120a79120bb 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -811,15 +811,24 @@ class TokenSearcher : public StringSearcherBase size_t needle_size; public: + + template + requires (sizeof(CharT) == 1) + static bool isValidNeedle(const CharT * needle_, size_t needle_size_) + { + if (std::any_of(needle_, needle_ + needle_size_, isTokenSeparator)) + return false; + return true; + } + template requires (sizeof(CharT) == 1) TokenSearcher(const CharT * needle_, size_t needle_size_) : searcher(needle_, needle_size_) , needle_size(needle_size_) { - if (std::any_of(needle_, needle_ + needle_size_, isTokenSeparator)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters"); - + /// The caller is responsible for calling isValidNeedle() + chassert(isValidNeedle(needle_, needle_size_)); } template @@ -880,6 +889,7 @@ using ASCIICaseSensitiveStringSearcher = impl::StringSearcher; using ASCIICaseInsensitiveStringSearcher = impl::StringSearcher; using UTF8CaseSensitiveStringSearcher = impl::StringSearcher; using UTF8CaseInsensitiveStringSearcher = impl::StringSearcher; + using ASCIICaseSensitiveTokenSearcher = impl::TokenSearcher; using ASCIICaseInsensitiveTokenSearcher = impl::TokenSearcher; diff --git a/src/Common/Volnitsky.h b/src/Common/Volnitsky.h index 64c5bf4c0d3..8f9aa23a38a 100644 --- a/src/Common/Volnitsky.h +++ b/src/Common/Volnitsky.h @@ -406,8 +406,7 @@ public: /// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first. for (auto i = static_cast(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i) { - bool ok = VolnitskyTraits::putNGram( - needle + i, static_cast(i + 1), needle, needle_size, callback); + bool ok = VolnitskyTraits::putNGram(needle + i, static_cast(i + 1), needle, needle_size, callback); /** `putNGramUTF8CaseInsensitive` does not work if characters with lower and upper cases * are represented by different number of bytes or code points. diff --git a/src/Functions/HasTokenImpl.h b/src/Functions/HasTokenImpl.h index 18e629b58fb..8cacdfff99d 100644 --- a/src/Functions/HasTokenImpl.h +++ b/src/Functions/HasTokenImpl.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include @@ -9,6 +10,7 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -44,51 +46,47 @@ struct HasTokenImpl const UInt8 * const end = haystack_data.data() + haystack_data.size(); const UInt8 * pos = begin; - try + if (!ASCIICaseSensitiveTokenSearcher::isValidNeedle(pattern.data(), pattern.size())) { - /// Parameter `pattern` is supposed to be a literal of letters and/or numbers. - /// Otherwise, an exception from the constructor of `TokenSearcher` is thrown. - /// If no exception is thrown at that point, then no further error cases may occur. - TokenSearcher searcher(pattern.data(), pattern.size(), end - pos); if (res_null) - std::ranges::fill(res_null->getData(), false); - - /// The current index in the array of strings. - size_t i = 0; - /// We will search for the next occurrence in all rows at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Let's determine which index it refers to. - while (begin + haystack_offsets[i] <= pos) - { - res[i] = negate; - ++i; - } - - /// We check that the entry does not pass through the boundaries of strings. - if (pos + pattern.size() < begin + haystack_offsets[i]) - res[i] = !negate; - else - res[i] = negate; - - pos = begin + haystack_offsets[i]; - ++i; - } - - /// Tail, in which there can be no substring. - if (i < res.size()) - memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); - } - catch (...) - { - if (!res_null) - throw; - else { std::ranges::fill(res, 0); std::ranges::fill(res_null->getData(), true); + return; } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters"); } + + TokenSearcher searcher(pattern.data(), pattern.size(), end - pos); + if (res_null) + std::ranges::fill(res_null->getData(), false); + + /// The current index in the array of strings. + size_t i = 0; + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) + { + /// Let's determine which index it refers to. + while (begin + haystack_offsets[i] <= pos) + { + res[i] = negate; + ++i; + } + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + pattern.size() < begin + haystack_offsets[i]) + res[i] = !negate; + else + res[i] = negate; + + pos = begin + haystack_offsets[i]; + ++i; + } + + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); } template diff --git a/src/Functions/hasToken.cpp b/src/Functions/hasToken.cpp index de67e80e752..b90750ea233 100644 --- a/src/Functions/hasToken.cpp +++ b/src/Functions/hasToken.cpp @@ -16,18 +16,18 @@ struct NameHasTokenOrNull static constexpr auto name = "hasTokenOrNull"; }; -using FunctionHasToken = DB::FunctionsStringSearch>; -using FunctionHasTokenOrNull = DB:: - FunctionsStringSearch, DB::ExecutionErrorPolicy::Null>; +using FunctionHasToken + = FunctionsStringSearch>; +using FunctionHasTokenOrNull + = FunctionsStringSearch, ExecutionErrorPolicy::Null>; REGISTER_FUNCTION(HasToken) { factory.registerFunction(FunctionDocumentation - {.description="Performs lookup of needle in haystack using tokenbf_v1 index."}, DB::FunctionFactory::CaseSensitive); + {.description="Performs lookup of needle in haystack using tokenbf_v1 index."}, FunctionFactory::CaseSensitive); factory.registerFunction(FunctionDocumentation - {.description="Performs lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."}, - DB::FunctionFactory::CaseSensitive); + {.description="Performs lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."}, FunctionFactory::CaseSensitive); } } diff --git a/src/Functions/hasTokenCaseInsensitive.cpp b/src/Functions/hasTokenCaseInsensitive.cpp index a6e8ecf3e9d..d7381e336b5 100644 --- a/src/Functions/hasTokenCaseInsensitive.cpp +++ b/src/Functions/hasTokenCaseInsensitive.cpp @@ -17,10 +17,9 @@ struct NameHasTokenCaseInsensitiveOrNull }; using FunctionHasTokenCaseInsensitive - = DB::FunctionsStringSearch>; -using FunctionHasTokenCaseInsensitiveOrNull = DB::FunctionsStringSearch< - DB::HasTokenImpl, - DB::ExecutionErrorPolicy::Null>; + = FunctionsStringSearch>; +using FunctionHasTokenCaseInsensitiveOrNull + = FunctionsStringSearch, ExecutionErrorPolicy::Null>; REGISTER_FUNCTION(HasTokenCaseInsensitive) { From 5de8c4ac679c0c79f0554681a3113be4a5c000d7 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 26 Jun 2023 15:27:05 +0000 Subject: [PATCH 142/171] Automatic style fix --- tests/integration/test_tlsv1_3/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_tlsv1_3/test.py b/tests/integration/test_tlsv1_3/test.py index a29dcb5fc1c..f5c2be51ed7 100644 --- a/tests/integration/test_tlsv1_3/test.py +++ b/tests/integration/test_tlsv1_3/test.py @@ -204,7 +204,9 @@ def test_https_non_ssl_auth(): err_str = str(err.value) if count < MAX_RETRY and "Broken pipe" in err_str: count = count + 1 - logging.warning(f"Failed attempt with wrong cert, user: peter, err: {err_str}") + logging.warning( + f"Failed attempt with wrong cert, user: peter, err: {err_str}" + ) continue assert "unknown ca" in err_str break @@ -222,7 +224,9 @@ def test_https_non_ssl_auth(): err_str = str(err.value) if count < MAX_RETRY and "Broken pipe" in err_str: count = count + 1 - logging.warning(f"Failed attempt with wrong cert, user: jane, err: {err_str}") + logging.warning( + f"Failed attempt with wrong cert, user: jane, err: {err_str}" + ) continue assert "unknown ca" in err_str break From a1eb27ce3038df0921d6667e65074168776988dc Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Jun 2023 18:12:41 +0200 Subject: [PATCH 143/171] Update log message --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 28 ++++++++++++------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index 76d54f9d27c..b65a368b936 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -878,10 +878,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() LOG_TEST( log, - "Current read type: {}, read offset: {}, impl offset: {}, file segment: {}", + "Current read type: {}, read offset: {}, impl offset: {}, impl position: {}, file segment: {}", toString(read_type), file_offset_of_buffer_end, implementation_buffer->getFileOffsetOfBufferEnd(), + implementation_buffer->getPosition(), file_segment.getInfoForLog()); chassert(current_read_range.left <= file_offset_of_buffer_end); @@ -940,7 +941,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() // We don't support implementation_buffer implementations that use nextimpl_working_buffer_offset. chassert(implementation_buffer->position() == implementation_buffer->buffer().begin()); - size = implementation_buffer->buffer().size(); + if (result) + size = implementation_buffer->buffer().size(); LOG_TEST( log, @@ -954,15 +956,21 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size); ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed); - const size_t new_file_offset = file_offset_of_buffer_end + size; - const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true); - if (new_file_offset > file_segment.range().right + 1 || new_file_offset > file_segment_write_offset) + if (result) { - auto file_segment_path = file_segment.getPathInLocalCache(); - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Read unexpected size. File size: {}, file path: {}, file segment info: {}", - fs::file_size(file_segment_path), file_segment_path, file_segment.getInfoForLog()); + const size_t new_file_offset = file_offset_of_buffer_end + size; + const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true); + if (new_file_offset > file_segment.range().right + 1 || new_file_offset > file_segment_write_offset) + { + auto file_segment_path = file_segment.getPathInLocalCache(); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Read unexpected size. " + "File size: {}, file segment path: {}, impl size: {}, impl path: {}" + "file segment info: {}", + fs::file_size(file_segment_path), file_segment_path, + implementation_buffer->getFileSize(), implementation_buffer->getFileName(), + file_segment.getInfoForLog()); + } } } else From d8cee8a34c4f3376b64977e222f46bd6abf0ffd5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 19:00:25 +0200 Subject: [PATCH 144/171] Fix style --- src/Common/StringSearcher.h | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index 120a79120bb..24b707c97ae 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -21,12 +21,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - - /** Variants for searching a substring in a string. * In most cases, performance is less than Volnitsky (see Volnitsky.h). */ From 1ed33b193628dd69ccfefb7ca343066608399bbb Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 19:22:46 +0200 Subject: [PATCH 145/171] Update src/Common/StringSearcher.h Co-authored-by: ltrk2 <107155950+ltrk2@users.noreply.github.com> --- src/Common/StringSearcher.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index 24b707c97ae..ff32218b64b 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -810,9 +810,7 @@ public: requires (sizeof(CharT) == 1) static bool isValidNeedle(const CharT * needle_, size_t needle_size_) { - if (std::any_of(needle_, needle_ + needle_size_, isTokenSeparator)) - return false; - return true; + return std::none_of(needle_, needle_ + needle_size_, isTokenSeparator)); } template From 61136680e403ae88469f66157e847168996be786 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 26 Jun 2023 20:05:58 +0200 Subject: [PATCH 146/171] Update src/Common/StringSearcher.h Co-authored-by: ltrk2 <107155950+ltrk2@users.noreply.github.com> --- src/Common/StringSearcher.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/StringSearcher.h b/src/Common/StringSearcher.h index ff32218b64b..3ed192d05f3 100644 --- a/src/Common/StringSearcher.h +++ b/src/Common/StringSearcher.h @@ -810,7 +810,7 @@ public: requires (sizeof(CharT) == 1) static bool isValidNeedle(const CharT * needle_, size_t needle_size_) { - return std::none_of(needle_, needle_ + needle_size_, isTokenSeparator)); + return std::none_of(needle_, needle_ + needle_size_, isTokenSeparator); } template From 01607f0b32291f75d86e55d17a9baa0ef82c1075 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Jun 2023 20:21:25 +0200 Subject: [PATCH 147/171] Add check for intersecting ranges --- src/Interpreters/Cache/FileCache.cpp | 24 +++++++++++++++++++----- src/Interpreters/Cache/FileSegment.h | 2 ++ src/Interpreters/Cache/Metadata.cpp | 22 ++++++++++++++++++++++ src/Interpreters/Cache/Metadata.h | 2 ++ 4 files changed, 45 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index b601c4029f3..9bd1bc503ff 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -499,12 +499,14 @@ KeyMetadata::iterator FileCache::addFileSegment( chassert(size > 0); /// Empty file segments in cache are not allowed. const auto & key = locked_key.getKey(); - if (locked_key.tryGetByOffset(offset)) + const FileSegment::Range range(offset, offset + size - 1); + + if (auto intersecting_range = locked_key.hasIntersectingRange(range)) { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Cache entry already exists for key: `{}`, offset: {}, size: {}.", - key, offset, size); + "Attempt to add intersecting file segment in cache ({} intersects {})", + range.toString(), intersecting_range->toString()); } FileSegment::State result_state; @@ -964,8 +966,20 @@ void FileCache::loadMetadata() if ((main_priority->getSizeLimit() == 0 || main_priority->getSize(lock) + size <= main_priority->getSizeLimit()) && (main_priority->getElementsLimit() == 0 || main_priority->getElementsCount(lock) + 1 <= main_priority->getElementsLimit())) { - auto file_segment_metadata_it = addFileSegment( - *locked_key, offset, size, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings(segment_kind), &lock); + KeyMetadata::iterator file_segment_metadata_it; + try + { + file_segment_metadata_it = addFileSegment( + *locked_key, offset, size, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings(segment_kind), &lock); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + chassert(false); + + fs::remove(offset_it->path()); + continue; + } const auto & file_segment_metadata = file_segment_metadata_it->second; chassert(file_segment_metadata->file_segment->assertCorrectness()); diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index 681c0d719e4..75053ff5dad 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -134,6 +134,8 @@ public: bool operator==(const Range & other) const { return left == other.left && right == other.right; } + bool operator<(const Range & other) const { return right < other.left; } + size_t size() const { return right - left + 1; } String toString() const { return fmt::format("[{}, {}]", std::to_string(left), std::to_string(right)); } diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index f799bae1e10..5be3bb6490a 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -451,6 +451,28 @@ void LockedKey::shrinkFileSegmentToDownloadedSize( chassert(file_segment->assertCorrectnessUnlocked(segment_lock)); } +std::optional LockedKey::hasIntersectingRange(const FileSegment::Range & range) const +{ + if (key_metadata->empty()) + return {}; + + auto it = key_metadata->lower_bound(range.left); + if (it == key_metadata->end()) + { + auto check_range = std::prev(it)->second->file_segment->range(); + return check_range < range ? std::nullopt : std::optional(check_range); + } + + std::optional check_range; + if (range < (check_range = it->second->file_segment->range()) + && (it == key_metadata->begin() || (check_range = std::prev(it)->second->file_segment->range()) < range)) + { + return {}; + } + + return check_range; +} + std::shared_ptr LockedKey::getByOffset(size_t offset) const { auto it = key_metadata->find(offset); diff --git a/src/Interpreters/Cache/Metadata.h b/src/Interpreters/Cache/Metadata.h index 635594a289a..8ee40aa977f 100644 --- a/src/Interpreters/Cache/Metadata.h +++ b/src/Interpreters/Cache/Metadata.h @@ -164,6 +164,8 @@ struct LockedKey : private boost::noncopyable bool isLastOwnerOfFileSegment(size_t offset) const; + std::optional hasIntersectingRange(const FileSegment::Range & range) const; + void removeFromCleanupQueue(); void markAsRemoved(); From 72e407eb06e506a03f3d491e81cf9b8982b469c0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 26 Jun 2023 20:40:18 +0200 Subject: [PATCH 148/171] Better --- src/Interpreters/Cache/Metadata.cpp | 21 +++++++++++---------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/Cache/Metadata.cpp b/src/Interpreters/Cache/Metadata.cpp index 5be3bb6490a..cde11a071f5 100644 --- a/src/Interpreters/Cache/Metadata.cpp +++ b/src/Interpreters/Cache/Metadata.cpp @@ -457,20 +457,21 @@ std::optional LockedKey::hasIntersectingRange(const FileSegm return {}; auto it = key_metadata->lower_bound(range.left); - if (it == key_metadata->end()) + if (it != key_metadata->end()) /// has next range { - auto check_range = std::prev(it)->second->file_segment->range(); - return check_range < range ? std::nullopt : std::optional(check_range); + auto next_range = it->second->file_segment->range(); + if (!(range < next_range)) + return next_range; + + if (it == key_metadata->begin()) + return {}; } - std::optional check_range; - if (range < (check_range = it->second->file_segment->range()) - && (it == key_metadata->begin() || (check_range = std::prev(it)->second->file_segment->range()) < range)) - { - return {}; - } + auto prev_range = std::prev(it)->second->file_segment->range(); + if (!(prev_range < range)) + return prev_range; - return check_range; + return {}; } std::shared_ptr LockedKey::getByOffset(size_t offset) const From c01d5ed004bbc64f95a7bfbacba1bda3bf2aa273 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 27 Jun 2023 00:27:51 +0200 Subject: [PATCH 149/171] Fix after review. --- src/Interpreters/ProcessList.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index c8e33b477a1..daa8d434ab6 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -593,10 +593,13 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even res.profile_counters = std::make_shared(thread_group->performance_counters.getPartiallyAtomicSnapshot()); } - if (get_settings && getContext()) + if (get_settings) { - res.query_settings = std::make_shared(getContext()->getSettings()); - res.current_database = getContext()->getCurrentDatabase(); + if (auto ctx = context.lock()) + { + res.query_settings = std::make_shared(ctx->getSettings()); + res.current_database = ctx->getCurrentDatabase(); + } } return res; From bcb106e1385b55b046643b55bb6683a666db81e4 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Mon, 26 Jun 2023 19:39:47 -0400 Subject: [PATCH 150/171] add missing aggregate functions (#51443) * add missing aggregate functions * add sparkBar --- .../aggregate-functions/reference/index.md | 20 +++++++++++++++++++ .../aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 21 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index 17ef494e9ad..6c56aefd51d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -19,8 +19,19 @@ Standard aggregate functions: - [stddevSamp](/docs/en/sql-reference/aggregate-functions/reference/stddevsamp.md) - [varPop](/docs/en/sql-reference/aggregate-functions/reference/varpop.md) - [varSamp](/docs/en/sql-reference/aggregate-functions/reference/varsamp.md) +- [corr](./corr.md) - [covarPop](/docs/en/sql-reference/aggregate-functions/reference/covarpop.md) - [covarSamp](/docs/en/sql-reference/aggregate-functions/reference/covarsamp.md) +- [entropy](./entropy.md) +- [exponentialMovingAverage](./exponentialmovingaverage.md) +- [intervalLengthSum](./intervalLengthSum.md) +- [kolmogorovSmirnovTest](./kolmogorovsmirnovtest.md) +- [mannwhitneyutest](./mannwhitneyutest.md) +- [median](./median.md) +- [rankCorr](./rankCorr.md) +- [sumKahan](./sumkahan.md) +- [studentTTest](./studentttest.md) +- [welchTTest](./welchttest.md) ClickHouse-specific aggregate functions: @@ -34,12 +45,15 @@ ClickHouse-specific aggregate functions: - [avgWeighted](/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md) - [topK](/docs/en/sql-reference/aggregate-functions/reference/topk.md) - [topKWeighted](/docs/en/sql-reference/aggregate-functions/reference/topkweighted.md) +- [deltaSum](./deltasum.md) +- [deltaSumTimestamp](./deltasumtimestamp.md) - [groupArray](/docs/en/sql-reference/aggregate-functions/reference/grouparray.md) - [groupArrayLast](/docs/en/sql-reference/aggregate-functions/reference/grouparraylast.md) - [groupUniqArray](/docs/en/sql-reference/aggregate-functions/reference/groupuniqarray.md) - [groupArrayInsertAt](/docs/en/sql-reference/aggregate-functions/reference/grouparrayinsertat.md) - [groupArrayMovingAvg](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingavg.md) - [groupArrayMovingSum](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingsum.md) +- [groupArraySample](./grouparraysample.md) - [groupBitAnd](/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md) - [groupBitOr](/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md) - [groupBitXor](/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md) @@ -84,3 +98,9 @@ ClickHouse-specific aggregate functions: - [theilsU](./theilsu.md) - [maxIntersections](./maxintersections.md) - [maxIntersectionsPosition](./maxintersectionsposition.md) +- [meanZTest](./meanztest.md) +- [quantileGK](./quantileGK.md) +- [quantileInterpolatedWeighted](./quantileinterpolatedweighted.md) +- [sparkBar](./sparkbar.md) +- [sumCount](./sumcount.md) + diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index bf988295644..00d047121e6 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -2169,6 +2169,7 @@ snowflakeToDateTime socketcache soundex sparkbar +sparkBar sparsehash speedscope splitByChar From 45f40cf2220fd7e37195fbed8d83db7f7cfa75e4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Jun 2023 04:16:26 +0200 Subject: [PATCH 151/171] Update system_warnings test --- tests/queries/0_stateless/01945_system_warnings.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01945_system_warnings.sh b/tests/queries/0_stateless/01945_system_warnings.sh index bf11cee2911..c9bd0fd4b8a 100755 --- a/tests/queries/0_stateless/01945_system_warnings.sh +++ b/tests/queries/0_stateless/01945_system_warnings.sh @@ -21,5 +21,4 @@ ${CLICKHOUSE_CLIENT} --multiple_joins_rewriter_version=42 -q "SELECT message FRO ${CLICKHOUSE_CLIENT} -q "SELECT count() = countDistinct(message) FROM system.warnings" # Avoid too many warnings, especially in CI -${CLICKHOUSE_CLIENT} -q "SELECT count() < 5 FROM system.warnings" - +${CLICKHOUSE_CLIENT} -q "SELECT count() < 10 FROM system.warnings" From 8c38d10bb4e838a156a2c1031cf2b60548e8cc42 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Jun 2023 05:49:03 +0200 Subject: [PATCH 152/171] A fix for clang-17 --- src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp index 4ad3cc7373a..188956b34fc 100644 --- a/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromMemoryStorageStep.cpp @@ -144,9 +144,9 @@ Pipe ReadFromMemoryStorageStep::makePipe() storage_snapshot, nullptr /* data */, nullptr /* parallel execution index */, - [storage = storage](std::shared_ptr & data_to_initialize) + [my_storage = storage](std::shared_ptr & data_to_initialize) { - data_to_initialize = static_cast(*storage).data.get(); + data_to_initialize = assert_cast(*my_storage).data.get(); })); } From 9d8b996f684ff5f669db49a616bd3ec3bf0e4c23 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 27 Jun 2023 04:30:56 +0000 Subject: [PATCH 153/171] Automatic style fix --- tests/ci/fast_test_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 7bb09d30177..2849759a3ee 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -150,7 +150,7 @@ def main(): os.makedirs(logs_path) run_log_path = os.path.join(logs_path, "run.log") - with TeePopen(run_cmd, run_log_path, timeout = 90 * 60) as process: + with TeePopen(run_cmd, run_log_path, timeout=90 * 60) as process: retcode = process.wait() if retcode == 0: logging.info("Run successfully") From 76108b955cdab459ba98dd660912f6e90aad57f4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 27 Jun 2023 07:06:50 +0200 Subject: [PATCH 154/171] Fix test 01605_adaptive_granularity_block_borders --- .../01605_adaptive_granularity_block_borders.sql | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql index ca7d0f3c950..4623c456475 100644 --- a/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql +++ b/tests/queries/0_stateless/01605_adaptive_granularity_block_borders.sql @@ -5,9 +5,9 @@ SET allow_prefetched_read_pool_for_remote_filesystem=0; DROP TABLE IF EXISTS adaptive_table; ---- If granularity of consequent blocks differs a lot, then adaptive ---- granularity will adjust amout of marks correctly. Data for test empirically ---- derived, it's quite hard to get good parameters. +-- If granularity of consequent blocks differs a lot, then adaptive +-- granularity will adjust the amount of marks correctly. +-- Data for test was empirically derived, it's quite hard to get good parameters. CREATE TABLE adaptive_table( key UInt64, @@ -32,6 +32,7 @@ SET enable_filesystem_cache = 0; -- If we have computed granularity incorrectly than we will exceed this limit. SET max_memory_usage='30M'; +SET max_threads = 1; SELECT max(length(value)) FROM adaptive_table; From 46a69829ff5a50867c21013b2ae9d28f2331876e Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 15:04:47 +0200 Subject: [PATCH 155/171] Delay shutdown of system and temporary databases --- src/Interpreters/DatabaseCatalog.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 8d3fa91a7fe..9a96bd09d82 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -216,8 +216,22 @@ void DatabaseCatalog::shutdownImpl() /// We still hold "databases" (instead of std::move) for Buffer tables to flush data correctly. + /// Delay shutdown of temporary and system databases. They will be shutdown last. + std::vector databases_with_delayed_shutdown; for (auto & database : current_databases) + { + if (database.first == TEMPORARY_DATABASE || database.first == SYSTEM_DATABASE) + { + databases_with_delayed_shutdown.push_back(database.second); + continue; + } database.second->shutdown(); + } + + for (auto & database : databases_with_delayed_shutdown) + { + database->shutdown(); + } { std::lock_guard lock(tables_marked_dropped_mutex); From 89de7a64eb3d36c3cbfed7388338e7352d216c78 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 27 Jun 2023 15:06:31 +0200 Subject: [PATCH 156/171] Fix memory leakage in CompressionCodecDeflateQpl. --- src/Compression/CompressionCodecDeflateQpl.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 4a2866d152d..859a9c2463a 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -9,6 +9,7 @@ #include #include "libaccel_config.h" #include +#include namespace DB { @@ -34,6 +35,7 @@ DeflateQplJobHWPool::DeflateQplJobHWPool() // loop all configured workqueue size to get maximum job number. accfg_ctx * ctx_ptr = nullptr; auto ctx_status = accfg_new(&ctx_ptr); + SCOPE_EXIT({ accfg_unref(ctx_ptr); }); if (ctx_status == 0) { auto * dev_ptr = accfg_device_get_first(ctx_ptr); From ddca9ece501e7e5554fbf439008f990997c7f2d1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 27 Jun 2023 15:32:14 +0200 Subject: [PATCH 157/171] Increase retries --- tests/integration/test_multiple_disks/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 54e7f6dd8ee..b5606ee8bc2 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -846,7 +846,7 @@ def test_start_stop_moves(start_cluster, name, engine): node1.query("SYSTEM START MOVES {}".format(name)) # wait sometime until background backoff finishes - retry = 30 + retry = 60 i = 0 while not sum(1 for x in used_disks if x == "jbod1") <= 2 and i < retry: time.sleep(1) From 99c9830737941e133814c75cd811ce4c5576b4fa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 27 Jun 2023 13:34:58 +0000 Subject: [PATCH 158/171] Docs: Add more blog posts to join docs --- docs/en/sql-reference/statements/select/join.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 08ffae838f8..7971b3ba275 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -21,6 +21,9 @@ Expressions from `ON` clause and columns from `USING` clause are called “join ## Related Content - Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Part 1](https://clickhouse.com/blog/clickhouse-fully-supports-joins) +- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 2](https://clickhouse.com/blog/clickhouse-fully-supports-joins-hash-joins-part2) +- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 3](https://clickhouse.com/blog/clickhouse-fully-supports-joins-full-sort-partial-merge-part3) +- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 4](https://clickhouse.com/blog/clickhouse-fully-supports-joins-direct-join-part4) ## Supported Types of JOIN From 5a39960e0374f355b22260bea1095676b50e92c6 Mon Sep 17 00:00:00 2001 From: Nikifor Seriakov Date: Tue, 27 Jun 2023 21:32:39 +0400 Subject: [PATCH 159/171] Update docs/en/interfaces/formats.md Fixed RawBLOB comparison lists formatting. --- docs/en/interfaces/formats.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 79baf04d75d..378a1c46d93 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -2454,18 +2454,22 @@ In this format, all input data is read to a single value. It is possible to pars The result is output in binary format without delimiters and escaping. If more than one value is output, the format is ambiguous, and it will be impossible to read the data back. Below is a comparison of the formats `RawBLOB` and [TabSeparatedRaw](#tabseparatedraw). + `RawBLOB`: - data is output in binary format, no escaping; - there are no delimiters between values; - no newline at the end of each value. -[TabSeparatedRaw] (#tabseparatedraw): + +`TabSeparatedRaw`: - data is output without escaping; - the rows contain values separated by tabs; - there is a line feed after the last value in every row. The following is a comparison of the `RawBLOB` and [RowBinary](#rowbinary) formats. + `RawBLOB`: - String fields are output without being prefixed by length. + `RowBinary`: - String fields are represented as length in varint format (unsigned [LEB128] (https://en.wikipedia.org/wiki/LEB128)), followed by the bytes of the string. From 04a08b47f3ca5d250600775adf1c307c09af929b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 22:25:56 +0200 Subject: [PATCH 160/171] Reorganize installation of clickhouse to have separated layers --- docker/server/Dockerfile.ubuntu | 46 +++++++++++++++++++++------------ 1 file changed, 30 insertions(+), 16 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index f393b98cfe6..abb3f387330 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -11,14 +11,15 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list && apt-get update \ && apt-get upgrade -yq \ && apt-get install --yes --no-install-recommends \ - apt-transport-https \ ca-certificates \ - dirmngr \ - gnupg2 \ - wget \ locales \ tzdata \ - && apt-get clean + wget \ + && apt-get clean \ + && rm -rf \ + /var/lib/apt/lists/* \ + /var/cache/debconf \ + /tmp/* ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" @@ -43,7 +44,8 @@ ARG single_binary_location_url="" ARG TARGETARCH -RUN arch=${TARGETARCH:-amd64} \ +# install from a web location with deb packages +RUN arch="${TARGETARCH:-amd64}" \ && if [ -n "${deb_location_url}" ]; then \ echo "installing from custom url with deb packages: ${deb_location_url}" \ rm -rf /tmp/clickhouse_debs \ @@ -54,15 +56,27 @@ RUN arch=${TARGETARCH:-amd64} \ || exit 1 \ ; done \ && dpkg -i /tmp/clickhouse_debs/*.deb ; \ - elif [ -n "${single_binary_location_url}" ]; then \ + fi + +# install from a single binary +RUN if [ -n "${single_binary_location_url}" ]; then \ echo "installing from single binary url: ${single_binary_location_url}" \ && rm -rf /tmp/clickhouse_binary \ && mkdir -p /tmp/clickhouse_binary \ && wget --progress=bar:force:noscroll "${single_binary_location_url}" -O /tmp/clickhouse_binary/clickhouse \ && chmod +x /tmp/clickhouse_binary/clickhouse \ && /tmp/clickhouse_binary/clickhouse install --user "clickhouse" --group "clickhouse" ; \ - else \ - mkdir -p /etc/apt/sources.list.d \ + fi + +# A fallback to installation from ClickHouse repository +RUN if ! clickhouse local -q "SELECT ''" > /dev/null; then \ + apt-get update \ + && apt-get install --yes --no-install-recommends \ + apt-transport-https \ + ca-certificates \ + dirmngr \ + gnupg2 \ + && mkdir -p /etc/apt/sources.list.d \ && apt-key adv --keyserver keyserver.ubuntu.com --recv 8919F6BD2B48D754 \ && echo ${REPOSITORY} > /etc/apt/sources.list.d/clickhouse.list \ && echo "installing from repository: ${REPOSITORY}" \ @@ -72,20 +86,20 @@ RUN arch=${TARGETARCH:-amd64} \ packages="${packages} ${package}=${VERSION}" \ ; done \ && apt-get install --allow-unauthenticated --yes --no-install-recommends ${packages} || exit 1 \ - ; fi \ - && clickhouse-local -q 'SELECT * FROM system.build_options' \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ /tmp/* \ - && mkdir -p /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client \ - && chmod ugo+Xrw -R /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client - -RUN apt-get autoremove --purge -yq libksba8 && \ - apt-get autoremove -yq + && apt-get autoremove --purge -yq libksba8 \ + && apt-get autoremove -yq \ + ; fi +# post install # we need to allow "others" access to clickhouse folder, because docker container # can be started with arbitrary uid (openshift usecase) +RUN clickhouse-local -q 'SELECT * FROM system.build_options' \ + && mkdir -p /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client \ + && chmod ugo+Xrw -R /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client RUN locale-gen en_US.UTF-8 ENV LANG en_US.UTF-8 From 6145baade8dad30207935c81a844fadd618cb912 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 22:27:33 +0200 Subject: [PATCH 161/171] Upgrade OS only once The ubuntu images are updated twice a month, it makes sense to do upgrade https://github.com/docker-library/official-images/commits/master/library/ubuntu --- docker/server/Dockerfile.ubuntu | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index abb3f387330..a563149ba8e 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -81,7 +81,6 @@ RUN if ! clickhouse local -q "SELECT ''" > /dev/null; then \ && echo ${REPOSITORY} > /etc/apt/sources.list.d/clickhouse.list \ && echo "installing from repository: ${REPOSITORY}" \ && apt-get update \ - && apt-get --yes -o "Dpkg::Options::=--force-confdef" -o "Dpkg::Options::=--force-confold" upgrade \ && for package in ${PACKAGES}; do \ packages="${packages} ${package}=${VERSION}" \ ; done \ From a2c4546c00f181ddc11a59c9e1b259819432e3c8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 22:53:30 +0200 Subject: [PATCH 162/171] Downgrade the base image to address issues on old dockerd --- docker/server/Dockerfile.ubuntu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index a563149ba8e..e2347945cf6 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -1,4 +1,4 @@ -FROM ubuntu:22.04 +FROM ubuntu:20.04 # see https://github.com/moby/moby/issues/4032#issuecomment-192327844 ARG DEBIAN_FRONTEND=noninteractive From 6d2b5166b930e77e53c9bb608d56d372c0ecd0b3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 23:10:56 +0200 Subject: [PATCH 163/171] Update golang version in builder --- docker/packager/binary/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index dd21c8552d3..e824161a688 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -89,7 +89,7 @@ RUN arch=${TARGETARCH:-amd64} \ && dpkg -i /tmp/nfpm.deb \ && rm /tmp/nfpm.deb -ARG GO_VERSION=1.19.5 +ARG GO_VERSION=1.19.10 # We need go for clickhouse-diagnostics RUN arch=${TARGETARCH:-amd64} \ && curl -Lo /tmp/go.tgz "https://go.dev/dl/go${GO_VERSION}.linux-${arch}.tar.gz" \ From 1cd5ae36bd89215d7c1af99d2a8b9193d0f79081 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 27 Jun 2023 23:20:47 +0200 Subject: [PATCH 164/171] Get rid of deprecated apt-key --- docker/server/Dockerfile.ubuntu | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index e2347945cf6..1bb0cfdc700 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -22,7 +22,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list /tmp/* ARG REPO_CHANNEL="stable" -ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" +ARG REPOSITORY="deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" ARG VERSION="23.5.3.24" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" @@ -77,8 +77,13 @@ RUN if ! clickhouse local -q "SELECT ''" > /dev/null; then \ dirmngr \ gnupg2 \ && mkdir -p /etc/apt/sources.list.d \ - && apt-key adv --keyserver keyserver.ubuntu.com --recv 8919F6BD2B48D754 \ - && echo ${REPOSITORY} > /etc/apt/sources.list.d/clickhouse.list \ + && GNUPGHOME=$(mktemp -d) \ + && GNUPGHOME="$GNUPGHOME" gpg --no-default-keyring \ + --keyring /usr/share/keyrings/clickhouse-keyring.gpg \ + --keyserver hkp://keyserver.ubuntu.com:80 --recv-keys 8919F6BD2B48D754 \ + && rm -r "$GNUPGHOME" \ + && chmod +r /usr/share/keyrings/clickhouse-keyring.gpg \ + && echo "${REPOSITORY}" > /etc/apt/sources.list.d/clickhouse.list \ && echo "installing from repository: ${REPOSITORY}" \ && apt-get update \ && for package in ${PACKAGES}; do \ From 67a7bdbb3a1c8fa2341e1627261d311cba1308a7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 28 Jun 2023 10:45:07 +0200 Subject: [PATCH 165/171] Rollback note about 22.04 in #50958 --- docker/server/README.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/server/README.md b/docker/server/README.md index 18dce492123..67646a262f5 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -20,7 +20,6 @@ For more information and documentation see https://clickhouse.com/. - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. - The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A). Most ARM CPUs after 2017 support ARMv8.2-A. A notable exception is Raspberry Pi 4 from 2019 whose CPU only supports ARMv8.0-A. -- Since the Clickhouse 23.3 Ubuntu image started using `ubuntu:22.04` as its base image, it requires docker version >= `20.10.10`, or use `docker run -- privileged` instead. Alternatively, try the Clickhouse Alpine image. ## How to use this image From 0c1f24db77354c7982eaac60618da3b15589e4a4 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 28 Jun 2023 12:13:16 +0200 Subject: [PATCH 166/171] Add a comment (#51517) * Add a comment * Update DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 40ef2bd4775..4cb2f6e3b3d 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -217,6 +217,8 @@ void DatabaseCatalog::shutdownImpl() /// We still hold "databases" (instead of std::move) for Buffer tables to flush data correctly. /// Delay shutdown of temporary and system databases. They will be shutdown last. + /// Because some databases might use them until their shutdown is called, but calling shutdown + /// on temporary database means clearing its set of tables, which will lead to unnecessary errors like "table not found". std::vector databases_with_delayed_shutdown; for (auto & database : current_databases) { From 0320ab4a7056ac099145648504505cb1c7225858 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 28 Jun 2023 12:16:16 +0200 Subject: [PATCH 167/171] Try making Keeper in `DatabaseReplicated` tests more stable (#51473) * Try making Keeper in DatabaseReplicated less flaky * lower log level to debug * Revert back to information --- tests/config/config.d/database_replicated.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 9a405f85908..2504a7ca526 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -40,7 +40,7 @@ 10000 30000 1000 - 4000 + 2000 5000 information false From b8b0fd3abfee08db6b31b393fb828915940ae13d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 28 Jun 2023 12:30:15 +0200 Subject: [PATCH 168/171] Slightly better code around packets for parallel replicas (#51451) --- src/Client/Connection.cpp | 8 +--- src/Client/IServerConnection.h | 10 +++-- src/QueryPipeline/RemoteQueryExecutor.cpp | 6 ++- src/Storages/MergeTree/MergeTreeReadPool.cpp | 21 ++++++---- src/Storages/MergeTree/MergeTreeReadPool.h | 18 ++++---- .../ParallelReplicasReadingCoordinator.cpp | 2 - src/Storages/MergeTree/RequestResponse.cpp | 26 +++++++++++- src/Storages/MergeTree/RequestResponse.h | 42 +++++++++++++++++-- 8 files changed, 98 insertions(+), 35 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index be226bfd9dd..cd102f46ffe 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -1175,16 +1175,12 @@ ProfileInfo Connection::receiveProfileInfo() const ParallelReadRequest Connection::receiveParallelReadRequest() const { - ParallelReadRequest request; - request.deserialize(*in); - return request; + return ParallelReadRequest::deserialize(*in); } InitialAllRangesAnnouncement Connection::receiveInitialParallelReadAnnounecement() const { - InitialAllRangesAnnouncement announcement; - announcement.deserialize(*in); - return announcement; + return InitialAllRangesAnnouncement::deserialize(*in); } diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 52382ff9d45..a0c029c79fb 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -16,6 +16,10 @@ #include +#include +#include +#include +#include namespace DB { @@ -34,9 +38,9 @@ struct Packet ProfileInfo profile_info; std::vector part_uuids; - InitialAllRangesAnnouncement announcement; - ParallelReadRequest request; - ParallelReadResponse response; + /// The part of parallel replicas protocol + std::optional announcement; + std::optional request; std::string server_timezone; diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 233dfe13339..cd6f65b7b43 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -434,11 +434,13 @@ RemoteQueryExecutor::ReadResult RemoteQueryExecutor::processPacket(Packet packet switch (packet.type) { case Protocol::Server::MergeTreeReadTaskRequest: - processMergeTreeReadTaskRequest(packet.request); + chassert(packet.request.has_value()); + processMergeTreeReadTaskRequest(packet.request.value()); return ReadResult(ReadResult::Type::ParallelReplicasToken); case Protocol::Server::MergeTreeAllRangesAnnounecement: - processMergeTreeInitialReadAnnounecement(packet.announcement); + chassert(packet.announcement.has_value()); + processMergeTreeInitialReadAnnounecement(packet.announcement.value()); return ReadResult(ReadResult::Type::ParallelReplicasToken); case Protocol::Server::ReadTaskRequest: diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index ba8c2c6385f..2ab90189f9d 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace ProfileEvents @@ -433,8 +434,12 @@ MergeTreeReadTaskPtr MergeTreeReadPoolParallelReplicas::getTask(size_t thread) if (buffered_ranges.empty()) { - auto result = extension.callback(ParallelReadRequest{ - .replica_num = extension.number_of_current_replica, .min_number_of_marks = min_marks_for_concurrent_read * threads}); + auto result = extension.callback(ParallelReadRequest( + CoordinationMode::Default, + extension.number_of_current_replica, + min_marks_for_concurrent_read * threads, + /// For Default coordination mode we don't need to pass part names. + RangesInDataPartsDescription{})); if (!result || result->finish) { @@ -529,12 +534,12 @@ MarkRanges MergeTreeInOrderReadPoolParallelReplicas::getNewTask(RangesInDataPart if (no_more_tasks) return {}; - auto response = extension.callback(ParallelReadRequest{ - .mode = mode, - .replica_num = extension.number_of_current_replica, - .min_number_of_marks = min_marks_for_concurrent_read * request.size(), - .description = request, - }); + auto response = extension.callback(ParallelReadRequest( + mode, + extension.number_of_current_replica, + min_marks_for_concurrent_read * request.size(), + request + )); if (!response || response->description.empty() || response->finish) { diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index 21273904e00..68d5438cb3d 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -193,10 +193,11 @@ public: predict_block_size_bytes, column_names, virtual_column_names, prewhere_info, actions_settings, reader_settings, per_part_params); - extension.all_callback({ - .description = parts_ranges.getDescriptions(), - .replica_num = extension.number_of_current_replica - }); + extension.all_callback(InitialAllRangesAnnouncement( + CoordinationMode::Default, + parts_ranges.getDescriptions(), + extension.number_of_current_replica + )); } ~MergeTreeReadPoolParallelReplicas() override; @@ -253,10 +254,11 @@ public: for (const auto & part : parts_ranges) buffered_tasks.push_back({part.data_part->info, MarkRanges{}}); - extension.all_callback({ - .description = parts_ranges.getDescriptions(), - .replica_num = extension.number_of_current_replica - }); + extension.all_callback(InitialAllRangesAnnouncement( + mode, + parts_ranges.getDescriptions(), + extension.number_of_current_replica + )); } MarkRanges getNewTask(RangesInDataPartDescription description); diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 2814d13cff0..57cd91cc995 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -102,7 +102,6 @@ public: explicit DefaultCoordinator(size_t replicas_count_) : ParallelReplicasReadingCoordinator::ImplInterface(replicas_count_) - , announcements(replicas_count_) , reading_state(replicas_count_) { } @@ -119,7 +118,6 @@ public: PartitionToBlockRanges partitions; size_t sent_initial_requests{0}; - std::vector announcements; Parts all_parts_to_read; /// Contains only parts which we haven't started to read from diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 05930d5a4c4..2ce0e20dcd2 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -51,7 +51,7 @@ String ParallelReadRequest::describe() const return result; } -void ParallelReadRequest::deserialize(ReadBuffer & in) +ParallelReadRequest ParallelReadRequest::deserialize(ReadBuffer & in) { UInt64 version; readIntBinary(version, in); @@ -60,12 +60,24 @@ void ParallelReadRequest::deserialize(ReadBuffer & in) "from replicas differ. Got: {}, supported version: {}", version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + CoordinationMode mode; + size_t replica_num; + size_t min_number_of_marks; + RangesInDataPartsDescription description; + uint8_t mode_candidate; readIntBinary(mode_candidate, in); mode = validateAndGet(mode_candidate); readIntBinary(replica_num, in); readIntBinary(min_number_of_marks, in); description.deserialize(in); + + return ParallelReadRequest( + mode, + replica_num, + min_number_of_marks, + std::move(description) + ); } void ParallelReadRequest::merge(ParallelReadRequest & other) @@ -125,7 +137,7 @@ String InitialAllRangesAnnouncement::describe() return result; } -void InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) +InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) { UInt64 version; readIntBinary(version, in); @@ -134,11 +146,21 @@ void InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) "from replicas differ. Got: {}, supported version: {}", version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + CoordinationMode mode; + RangesInDataPartsDescription description; + size_t replica_num; + uint8_t mode_candidate; readIntBinary(mode_candidate, in); mode = validateAndGet(mode_candidate); description.deserialize(in); readIntBinary(replica_num, in); + + return InitialAllRangesAnnouncement { + mode, + description, + replica_num + }; } } diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index 7e5563c0553..3a5bfde6c20 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -40,21 +40,40 @@ struct PartBlockRange } }; +/// ParallelReadRequest is used by remote replicas during parallel read +/// to signal an initiator that they need more marks to read. struct ParallelReadRequest { + /// No default constructor, you must initialize all fields at once. + + ParallelReadRequest( + CoordinationMode mode_, + size_t replica_num_, + size_t min_number_of_marks_, + RangesInDataPartsDescription description_) + : mode(mode_) + , replica_num(replica_num_) + , min_number_of_marks(min_number_of_marks_) + , description(std::move(description_)) + {} + CoordinationMode mode; size_t replica_num; size_t min_number_of_marks; - - /// Extension for ordered mode + /// Extension for Ordered (InOrder or ReverseOrder) mode + /// Contains only data part names without mark ranges. RangesInDataPartsDescription description; void serialize(WriteBuffer & out) const; String describe() const; - void deserialize(ReadBuffer & in); + static ParallelReadRequest deserialize(ReadBuffer & in); void merge(ParallelReadRequest & other); }; +/// ParallelReadResponse is used by an initiator to tell +/// remote replicas about what to read during parallel reading. +/// Additionally contains information whether there are more available +/// marks to read (whether it is the last packet or not). struct ParallelReadResponse { bool finish{false}; @@ -66,15 +85,30 @@ struct ParallelReadResponse }; +/// The set of parts (their names) along with ranges to read which is sent back +/// to the initiator by remote replicas during parallel reading. +/// Additionally contains an identifier (replica_num) plus +/// the reading algorithm chosen (Default, InOrder or ReverseOrder). struct InitialAllRangesAnnouncement { + /// No default constructor, you must initialize all fields at once. + + InitialAllRangesAnnouncement( + CoordinationMode mode_, + RangesInDataPartsDescription description_, + size_t replica_num_) + : mode(mode_) + , description(description_) + , replica_num(replica_num_) + {} + CoordinationMode mode; RangesInDataPartsDescription description; size_t replica_num; void serialize(WriteBuffer & out) const; String describe(); - void deserialize(ReadBuffer & in); + static InitialAllRangesAnnouncement deserialize(ReadBuffer & in); }; From 9c2a9a60ea9a1f474495d74c399df5ffe5dddb45 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 28 Jun 2023 16:35:41 +0000 Subject: [PATCH 169/171] Update version_date.tsv and changelogs after v23.3.6.7-lts --- docs/changelogs/v23.3.6.7-lts.md | 19 +++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 20 insertions(+) create mode 100644 docs/changelogs/v23.3.6.7-lts.md diff --git a/docs/changelogs/v23.3.6.7-lts.md b/docs/changelogs/v23.3.6.7-lts.md new file mode 100644 index 00000000000..387cc126aba --- /dev/null +++ b/docs/changelogs/v23.3.6.7-lts.md @@ -0,0 +1,19 @@ +--- +sidebar_position: 1 +sidebar_label: 2023 +--- + +# 2023 Changelog + +### ClickHouse release v23.3.6.7-lts (7e3f0a271b7) FIXME as compared to v23.3.5.9-lts (f5fbc2fd2b3) + +#### Improvement +* Backported in [#51240](https://github.com/ClickHouse/ClickHouse/issues/51240): Improve the progress bar for file/s3/hdfs/url table functions by using chunk size from source data and using incremental total size counting in each thread. Fix the progress bar for *Cluster functions. This closes [#47250](https://github.com/ClickHouse/ClickHouse/issues/47250). [#51088](https://github.com/ClickHouse/ClickHouse/pull/51088) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Build/Testing/Packaging Improvement +* Backported in [#51529](https://github.com/ClickHouse/ClickHouse/issues/51529): Split huge `RUN` in Dockerfile into smaller conditional. Install the necessary tools on demand in the same `RUN` layer, and remove them after that. Upgrade the OS only once at the beginning. Use a modern way to check the signed repository. Downgrade the base repo to ubuntu:20.04 to address the issues on older docker versions. Upgrade golang version to address golang vulnerabilities. [#51504](https://github.com/ClickHouse/ClickHouse/pull/51504) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Fix type of LDAP server params hash in cache entry [#50865](https://github.com/ClickHouse/ClickHouse/pull/50865) ([Julian Maicher](https://github.com/jmaicher)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index dd21cc7e953..307ed97068f 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -5,6 +5,7 @@ v23.4.4.16-stable 2023-06-17 v23.4.3.48-stable 2023-06-12 v23.4.2.11-stable 2023-05-02 v23.4.1.1943-stable 2023-04-27 +v23.3.6.7-lts 2023-06-28 v23.3.5.9-lts 2023-06-22 v23.3.4.17-lts 2023-06-17 v23.3.3.52-lts 2023-06-12 From 29a0220e63420a60d1aef9cfec0f9c6c0bba8160 Mon Sep 17 00:00:00 2001 From: Thom O'Connor Date: Wed, 28 Jun 2023 14:42:01 -0600 Subject: [PATCH 170/171] Update functions-for-nulls.md Fixing broken link to HTML anchor on page --- docs/en/sql-reference/functions/functions-for-nulls.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index 6f82fedaab7..d57b799e94c 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -8,7 +8,7 @@ sidebar_label: Nullable ## isNull -Returns whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). +Returns whether the argument is [NULL](../../sql-reference/syntax.md#null). ``` sql isNull(x) From 92d7b067b0ae9eff9b28fb41c8ffbeebc8b2c31c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 28 Jun 2023 23:36:15 +0200 Subject: [PATCH 171/171] Bring back lost /tmp cleanup in clickhouse-server docker image --- docker/server/Dockerfile.ubuntu | 22 ++++++++++++---------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 1bb0cfdc700..42ae81655d2 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -48,14 +48,15 @@ ARG TARGETARCH RUN arch="${TARGETARCH:-amd64}" \ && if [ -n "${deb_location_url}" ]; then \ echo "installing from custom url with deb packages: ${deb_location_url}" \ - rm -rf /tmp/clickhouse_debs \ + && rm -rf /tmp/clickhouse_debs \ && mkdir -p /tmp/clickhouse_debs \ && for package in ${PACKAGES}; do \ { wget --progress=bar:force:noscroll "${deb_location_url}/${package}_${VERSION}_${arch}.deb" -P /tmp/clickhouse_debs || \ wget --progress=bar:force:noscroll "${deb_location_url}/${package}_${VERSION}_all.deb" -P /tmp/clickhouse_debs ; } \ || exit 1 \ ; done \ - && dpkg -i /tmp/clickhouse_debs/*.deb ; \ + && dpkg -i /tmp/clickhouse_debs/*.deb \ + && rm -rf /tmp/* ; \ fi # install from a single binary @@ -65,11 +66,12 @@ RUN if [ -n "${single_binary_location_url}" ]; then \ && mkdir -p /tmp/clickhouse_binary \ && wget --progress=bar:force:noscroll "${single_binary_location_url}" -O /tmp/clickhouse_binary/clickhouse \ && chmod +x /tmp/clickhouse_binary/clickhouse \ - && /tmp/clickhouse_binary/clickhouse install --user "clickhouse" --group "clickhouse" ; \ + && /tmp/clickhouse_binary/clickhouse install --user "clickhouse" --group "clickhouse" \ + && rm -rf /tmp/* ; \ fi # A fallback to installation from ClickHouse repository -RUN if ! clickhouse local -q "SELECT ''" > /dev/null; then \ +RUN if ! clickhouse local -q "SELECT ''" > /dev/null 2>&1; then \ apt-get update \ && apt-get install --yes --no-install-recommends \ apt-transport-https \ @@ -90,12 +92,12 @@ RUN if ! clickhouse local -q "SELECT ''" > /dev/null; then \ packages="${packages} ${package}=${VERSION}" \ ; done \ && apt-get install --allow-unauthenticated --yes --no-install-recommends ${packages} || exit 1 \ - && rm -rf \ - /var/lib/apt/lists/* \ - /var/cache/debconf \ - /tmp/* \ - && apt-get autoremove --purge -yq libksba8 \ - && apt-get autoremove -yq \ + && rm -rf \ + /var/lib/apt/lists/* \ + /var/cache/debconf \ + /tmp/* \ + && apt-get autoremove --purge -yq libksba8 \ + && apt-get autoremove -yq \ ; fi # post install