Fix tests

This commit is contained in:
Amos Bird 2024-02-20 23:38:05 +08:00
parent c5e073320c
commit 7435568a0d
No known key found for this signature in database
GPG Key ID: 80D430DCBECFEDB4
36 changed files with 382 additions and 371 deletions

View File

@ -241,7 +241,7 @@ public:
CNF::OrGroup or_group;
visitImpl(node, or_group);
if (!or_group.empty())
and_group.insert(std::move(or_group));
and_group.push_back(std::move(or_group));
}
CNF::AndGroup and_group;
@ -254,7 +254,7 @@ private:
auto * function_node = node->as<FunctionNode>();
if (!function_node || !isLogicalFunction(*function_node))
{
or_group.insert(CNF::AtomicFormula{false, std::move(node)});
or_group.push_back(CNF::AtomicFormula{false, std::move(node)});
return;
}
@ -268,7 +268,7 @@ private:
CNF::OrGroup argument_or_group;
visitImpl(argument, argument_or_group);
if (!argument_or_group.empty())
and_group.insert(std::move(argument_or_group));
and_group.push_back(std::move(argument_or_group));
}
}
else if (name == "or")
@ -281,7 +281,7 @@ private:
{
assert(name == "not");
auto & arguments = function_node->getArguments().getNodes();
or_group.insert(CNF::AtomicFormula{true, std::move(arguments[0])});
or_group.push_back(CNF::AtomicFormula{true, std::move(arguments[0])});
}
}
};
@ -351,7 +351,7 @@ CNF & CNF::transformGroups(std::function<OrGroup(const OrGroup &)> fn)
{
auto new_group = fn(group);
if (!new_group.empty())
result.insert(std::move(new_group));
result.push_back(std::move(new_group));
}
statements = std::move(result);
@ -367,7 +367,7 @@ CNF & CNF::transformAtoms(std::function<AtomicFormula(const AtomicFormula &)> fn
{
auto new_atom = fn(atom);
if (new_atom.node_with_hash.node)
result.insert(std::move(new_atom));
result.push_back(std::move(new_atom));
}
return result;
@ -442,7 +442,7 @@ CNF & CNF::filterAlwaysTrueGroups(std::function<bool(const OrGroup &)> predicate
for (const auto & or_group : statements)
{
if (predicate(or_group))
filtered.insert(or_group);
filtered.push_back(or_group);
}
statements = std::move(filtered);
@ -458,16 +458,16 @@ CNF & CNF::filterAlwaysFalseAtoms(std::function<bool(const AtomicFormula &)> pre
for (const auto & atom : or_group)
{
if (predicate(atom))
filtered_group.insert(atom);
filtered_group.push_back(atom);
}
if (!filtered_group.empty())
filtered.insert(std::move(filtered_group));
filtered.push_back(std::move(filtered_group));
else
{
filtered.clear();
filtered_group.insert(AtomicFormula{false, QueryTreeNodePtrWithHash{std::make_shared<ConstantNode>(static_cast<UInt8>(0))}});
filtered.insert(std::move(filtered_group));
filtered_group.push_back(AtomicFormula{false, QueryTreeNodePtrWithHash{std::make_shared<ConstantNode>(static_cast<UInt8>(0))}});
filtered.push_back(std::move(filtered_group));
break;
}
}
@ -494,7 +494,7 @@ CNF & CNF::reduce()
void CNF::appendGroup(const AndGroup & and_group)
{
for (const auto & or_group : and_group)
statements.emplace(or_group);
statements.emplace_back(or_group);
}
CNF::CNF(AndGroup statements_)

View File

@ -6,8 +6,10 @@
#include <Common/SipHash.h>
#include <Interpreters/Context_fwd.h>
#include <unordered_set>
#include <boost/multi_index_container.hpp>
#include <boost/multi_index/sequenced_index.hpp>
#include <boost/multi_index/identity.hpp>
#include <boost/multi_index/ordered_index.hpp>
namespace DB::Analyzer
{
@ -15,6 +17,12 @@ namespace DB::Analyzer
class CNF
{
public:
template <typename T>
using CNFSet = boost::multi_index_container<
T,
boost::multi_index::
indexed_by<boost::multi_index::sequenced<>, boost::multi_index::ordered_unique<boost::multi_index::identity<T>>>>;
struct AtomicFormula
{
bool negative = false;
@ -24,9 +32,9 @@ public:
bool operator<(const AtomicFormula & rhs) const;
};
// Different hash is generated for different order, so we use std::set
using OrGroup = std::set<AtomicFormula>;
using AndGroup = std::set<OrGroup>;
/// Keep insertion order so that optimization result is stable
using OrGroup = CNFSet<AtomicFormula>;
using AndGroup = CNFSet<OrGroup>;
std::string dump() const;

View File

@ -286,7 +286,7 @@ Analyzer::CNF::OrGroup createIndexHintGroup(
helper_function_node.getArguments().getNodes()[index] = primary_key_node->clone();
auto reverse_function_name = getReverseRelationMap().at(mostStrict(expected_result, actual_result));
helper_function_node.resolveAsFunction(FunctionFactory::instance().get(reverse_function_name, context));
result.insert(Analyzer::CNF::AtomicFormula{atom.negative, std::move(helper_node)});
result.push_back(Analyzer::CNF::AtomicFormula{atom.negative, std::move(helper_node)});
return true;
}
}
@ -332,7 +332,7 @@ void addIndexConstraint(Analyzer::CNF & cnf, const QueryTreeNodes & table_expres
{
auto new_group = createIndexHintGroup(group, graph, primary_key_only_nodes, context);
if (!new_group.empty())
and_group.emplace(std::move(new_group));
and_group.emplace_back(std::move(new_group));
}
if (!and_group.empty())
@ -341,7 +341,7 @@ void addIndexConstraint(Analyzer::CNF & cnf, const QueryTreeNodes & table_expres
auto index_hint_node = std::make_shared<FunctionNode>("indexHint");
index_hint_node->getArguments().getNodes().push_back(Analyzer::CNF{std::move(and_group)}.toQueryTree());
index_hint_node->resolveAsFunction(FunctionFactory::instance().get("indexHint", context));
new_group.insert({false, QueryTreeNodePtrWithHash{std::move(index_hint_node)}});
new_group.push_back({false, QueryTreeNodePtrWithHash{std::move(index_hint_node)}});
cnf.appendGroup({new_group});
}

View File

@ -132,7 +132,7 @@ namespace
auto * helper_func = helper_ast->as<ASTFunction>();
helper_func->name = getReverseRelationMap().at(mostStrict(need_result, actual_result));
helper_func->arguments->children[index] = primary_key_ast->clone();
result.insert(CNFQuery::AtomicFormula{atom.negative, helper_ast});
result.push_back(CNFQuery::AtomicFormula{atom.negative, helper_ast});
return true;
}
}
@ -167,13 +167,14 @@ void AddIndexConstraintsOptimizer::perform(CNFQuery & cnf_query)
{
auto add_group = createIndexHintGroup(or_group, graph, primary_key_only_asts);
if (!add_group.empty())
and_group.emplace(std::move(add_group));
and_group.emplace_back(std::move(add_group));
});
if (!and_group.empty())
{
CNFQuery::OrGroup new_or_group;
new_or_group.insert(CNFQuery::AtomicFormula{false, makeASTFunction("indexHint", TreeCNFConverter::fromCNF(CNFQuery(std::move(and_group))))});
new_or_group.push_back(
CNFQuery::AtomicFormula{false, makeASTFunction("indexHint", TreeCNFConverter::fromCNF(CNFQuery(std::move(and_group))))});
cnf_query.appendGroup(CNFQuery::AndGroup{new_or_group});
}
}

View File

@ -177,7 +177,7 @@ void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery::
CNFQuery::OrGroup group;
traverseCNF(child, and_group, group);
if (!group.empty())
and_group.insert(std::move(group));
and_group.push_back(std::move(group));
}
}
else if (func && func->name == "or")
@ -191,11 +191,11 @@ void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery::
{
if (func->arguments->children.size() != 1)
throw Exception(ErrorCodes::INCORRECT_QUERY, "Bad NOT function. Expected 1 argument");
or_group.insert(CNFQuery::AtomicFormula{true, func->arguments->children.front()});
or_group.push_back(CNFQuery::AtomicFormula{true, func->arguments->children.front()});
}
else
{
or_group.insert(CNFQuery::AtomicFormula{false, node});
or_group.push_back(CNFQuery::AtomicFormula{false, node});
}
}
@ -204,7 +204,7 @@ void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & result)
CNFQuery::OrGroup or_group;
traverseCNF(node, result, or_group);
if (!or_group.empty())
result.insert(or_group);
result.push_back(or_group);
}
}

View File

@ -1,17 +1,25 @@
#pragma once
#include <set>
#include <unordered_map>
#include <vector>
#include <Parsers/ASTLiteral.h>
#include <Parsers/IAST_fwd.h>
#include <boost/multi_index_container.hpp>
#include <boost/multi_index/sequenced_index.hpp>
#include <boost/multi_index/identity.hpp>
#include <boost/multi_index/ordered_index.hpp>
namespace DB
{
class CNFQuery
{
public:
template <typename T>
using CNFSet = boost::multi_index_container<
T,
boost::multi_index::
indexed_by<boost::multi_index::sequenced<>, boost::multi_index::ordered_unique<boost::multi_index::identity<T>>>>;
struct AtomicFormula
{
bool negative = false;
@ -33,8 +41,8 @@ public:
}
};
using OrGroup = std::set<AtomicFormula>;
using AndGroup = std::set<OrGroup>;
using OrGroup = CNFSet<AtomicFormula>;
using AndGroup = CNFSet<OrGroup>;
CNFQuery(AndGroup && statements_) : statements(std::move(statements_)) { } /// NOLINT
@ -45,7 +53,7 @@ public:
for (const auto & or_group : statements)
{
if (predicate_is_unknown(or_group))
filtered.insert(or_group);
filtered.push_back(or_group);
}
std::swap(statements, filtered);
return *this;
@ -61,17 +69,17 @@ public:
for (auto ast : or_group)
{
if (predicate_is_unknown(ast))
filtered_group.insert(ast);
filtered_group.push_back(ast);
}
if (!filtered_group.empty())
filtered.insert(filtered_group);
filtered.push_back(filtered_group);
else
{
/// all atoms false -> group false -> CNF false
filtered.clear();
filtered_group.clear();
filtered_group.insert(AtomicFormula{false, std::make_shared<ASTLiteral>(static_cast<UInt8>(0))});
filtered.insert(filtered_group);
filtered_group.push_back(AtomicFormula{false, std::make_shared<ASTLiteral>(static_cast<UInt8>(0))});
filtered.push_back(filtered_group);
std::swap(statements, filtered);
return *this;
}
@ -91,7 +99,7 @@ public:
CNFQuery & appendGroup(AndGroup&& and_group)
{
for (auto && or_group : and_group)
statements.emplace(or_group);
statements.emplace_back(or_group);
return *this;
}
@ -103,7 +111,7 @@ public:
{
auto new_group = func(group);
if (!new_group.empty())
result.insert(std::move(new_group));
result.push_back(std::move(new_group));
}
std::swap(statements, result);
return *this;
@ -119,7 +127,7 @@ public:
{
auto new_atom = func(atom);
if (new_atom.ast)
result.insert(std::move(new_atom));
result.push_back(std::move(new_atom));
}
return result;
});
@ -175,25 +183,25 @@ TAndGroup reduceOnceCNFStatements(const TAndGroup & groups)
bool inserted = false;
for (const auto & atom : group)
{
copy.erase(atom);
copy.template get<1>().erase(atom);
using AtomType = std::decay_t<decltype(atom)>;
AtomType negative_atom(atom);
negative_atom.negative = !atom.negative;
copy.insert(negative_atom);
copy.push_back(negative_atom);
if (groups.contains(copy))
if (groups.template get<1>().contains(copy))
{
copy.erase(negative_atom);
result.insert(copy);
copy.template get<1>().erase(negative_atom);
result.push_back(copy);
inserted = true;
break;
}
copy.erase(negative_atom);
copy.insert(atom);
copy.template get<1>().erase(negative_atom);
copy.push_back(atom);
}
if (!inserted)
result.insert(group);
result.push_back(group);
}
return result;
}
@ -204,7 +212,7 @@ bool isCNFGroupSubset(const TOrGroup & left, const TOrGroup & right)
if (left.size() > right.size())
return false;
for (const auto & elem : left)
if (!right.contains(elem))
if (!right.template get<1>().contains(elem))
return false;
return true;
}
@ -227,7 +235,7 @@ TAndGroup filterCNFSubsets(const TAndGroup & groups)
}
if (insert)
result.insert(group);
result.push_back(group);
}
return result;
}

View File

@ -138,7 +138,7 @@ Block getHeaderForProcessingStage(
if (context->getSettingsRef().allow_experimental_analyzer)
{
auto storage = std::make_shared<StorageDummy>(
storage_snapshot->storage.getStorageID(), storage_snapshot->metadata->getColumns(), &storage_snapshot->storage);
storage_snapshot->storage.getStorageID(), storage_snapshot->metadata->getColumns(), storage_snapshot);
InterpreterSelectQueryAnalyzer interpreter(query, context, storage, SelectQueryOptions(processed_stage).analyze());
result = interpreter.getSampleBlock();
}

View File

@ -413,7 +413,7 @@ QueryTreeNodePtr replaceTableExpressionsWithDummyTables(const QueryTreeNodePtr &
storage_dummy = std::make_shared<StorageDummy>(
storage_snapshot->storage.getStorageID(),
ColumnsDescription(storage_snapshot->getColumns(get_column_options)),
&storage_snapshot->storage);
storage_snapshot);
}
else if (subquery_node || union_node)
{

View File

@ -129,7 +129,7 @@ public:
auto storage_dummy = std::make_shared<StorageDummy>(
storage_snapshot->storage.getStorageID(),
ColumnsDescription(storage_snapshot->getColumns(get_column_options)),
&storage_snapshot->storage);
storage_snapshot);
auto dummy_table_node = std::make_shared<TableNode>(std::move(storage_dummy), context);

View File

@ -46,9 +46,7 @@ void optimizePrewhere(Stack & stack, QueryPlan::Nodes &)
return;
const auto & storage_snapshot = source_step_with_filter->getStorageSnapshot();
const auto * maybe_dummy = typeid_cast<const StorageDummy *>(&storage_snapshot->storage);
const auto & storage = maybe_dummy && maybe_dummy->getOriginalStorage() ? *maybe_dummy->getOriginalStorage() : storage_snapshot->storage;
const auto & storage = storage_snapshot->storage;
if (!storage.canMoveConditionsToPrewhere())
return;

View File

@ -1766,18 +1766,6 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const
return *result_ptr;
}
bool ReadFromMergeTree::isQueryWithSampling() const
{
if (context->getSettingsRef().parallel_replicas_count > 1 && data.supportsSampling())
return true;
const auto & select = query_info.query->as<ASTSelectQuery &>();
if (query_info.table_expression_modifiers)
return query_info.table_expression_modifiers->getSampleSizeRatio() != std::nullopt;
else
return select.sampleSize() != nullptr;
}
Pipe ReadFromMergeTree::spreadMarkRanges(
RangesInDataParts && parts_with_ranges, size_t num_streams, AnalysisResult & result, ActionsDAGPtr & result_projection)
{
@ -1873,6 +1861,11 @@ Pipe ReadFromMergeTree::groupStreamsByPartition(AnalysisResult & result, Actions
void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
auto result = getAnalysisResult();
/// Do not keep data parts in snapshot.
/// They are stored separately, and some could be released after PK analysis.
storage_snapshot->data = std::make_unique<MergeTreeData::SnapshotData>();
result.checkLimits(context->getSettingsRef(), query_info);
LOG_DEBUG(

View File

@ -180,7 +180,6 @@ public:
bool readsInOrder() const;
void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value) override;
bool isQueryWithSampling() const override;
/// Returns true if the optimization is applicable (and applies it then).
bool requestOutputEachPartitionThroughSeparatePort();

View File

@ -3,6 +3,8 @@
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <IO/Operators.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTSelectQuery.h>
#include <Common/JSONBuilder.h>
namespace DB
@ -69,6 +71,18 @@ Block SourceStepWithFilter::applyPrewhereActions(Block block, const PrewhereInfo
return block;
}
bool SourceStepWithFilter::isQueryWithSampling() const
{
if (context->getSettingsRef().parallel_replicas_count > 1 && storage_snapshot->storage.supportsSampling())
return true;
const auto & select = query_info.query->as<ASTSelectQuery &>();
if (query_info.table_expression_modifiers)
return query_info.table_expression_modifiers->getSampleSizeRatio() != std::nullopt;
else
return select.sampleSize() != nullptr;
}
void SourceStepWithFilter::updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value)
{
query_info.prewhere_info = prewhere_info_value;

View File

@ -41,7 +41,7 @@ public:
const StorageSnapshotPtr & getStorageSnapshot() const { return storage_snapshot; }
bool isQueryWithFinal() const { return query_info.isFinal(); }
virtual bool isQueryWithSampling() const { return false; }
bool isQueryWithSampling() const;
const Names & requiredSourceColumns() const { return required_source_columns; }

View File

@ -1000,10 +1000,6 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
selectColumnNames(column_names_to_return, data, real_column_names, virt_column_names, sample_factor_column_queried);
/// Do not keep data parts in snapshot.
/// They are stored separately, and some could be released after PK analysis.
auto storage_snapshot_copy = storage_snapshot->clone(std::make_unique<MergeTreeData::SnapshotData>());
return std::make_unique<ReadFromMergeTree>(
std::move(parts),
std::move(alter_conversions),
@ -1012,7 +1008,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
virt_column_names,
data,
query_info,
storage_snapshot_copy,
storage_snapshot,
context,
max_block_size,
num_streams,

View File

@ -10,8 +10,9 @@
namespace DB
{
StorageDummy::StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_, StorageRawPtr original_storage_)
: IStorage(table_id_), original_storage(original_storage_)
StorageDummy::StorageDummy(
const StorageID & table_id_, const ColumnsDescription & columns_, const StorageSnapshotPtr & original_storage_snapshot_)
: IStorage(table_id_), original_storage_snapshot(original_storage_snapshot_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
@ -39,7 +40,7 @@ void StorageDummy::read(QueryPlan & query_plan,
query_plan.addStep(std::make_unique<ReadFromDummy>(
column_names,
query_info,
storage_snapshot,
original_storage_snapshot ? original_storage_snapshot : storage_snapshot,
local_context,
*this));
}

View File

@ -8,12 +8,11 @@
namespace DB
{
using StorageRawPtr = const IStorage *;
class StorageDummy final : public IStorage
{
public:
StorageDummy(const StorageID & table_id_, const ColumnsDescription & columns_, StorageRawPtr orignal_storage = nullptr);
StorageDummy(
const StorageID & table_id_, const ColumnsDescription & columns_, const StorageSnapshotPtr & original_storage_snapshot_ = nullptr);
std::string getName() const override { return "StorageDummy"; }
@ -22,7 +21,10 @@ public:
bool supportsPrewhere() const override { return true; }
bool supportsSubcolumns() const override { return true; }
bool supportsDynamicSubcolumns() const override { return true; }
bool canMoveConditionsToPrewhere() const override { return original_storage ? original_storage->canMoveConditionsToPrewhere() : false; }
bool canMoveConditionsToPrewhere() const override
{
return original_storage_snapshot ? original_storage_snapshot->storage.canMoveConditionsToPrewhere() : false;
}
StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr /*query_context*/) const override
{
@ -45,13 +47,13 @@ public:
size_t max_block_size,
size_t num_streams) override;
StorageRawPtr getOriginalStorage() const { return original_storage; }
const StorageSnapshotPtr & getOriginalStorageSnapshot() const { return original_storage_snapshot; }
private:
const ColumnsDescription object_columns;
/// The original storage which is replaced during planning. See collectFiltersForAnalysis.
StorageRawPtr original_storage;
/// The original storage snapshot which is replaced during planning. See collectFiltersForAnalysis for example.
StorageSnapshotPtr original_storage_snapshot;
};
class ReadFromDummy final : public SourceStepWithFilter

View File

@ -24,6 +24,6 @@ FROM
n,
finalizeAggregation(s)
FROM test_00808_push_down_with_finalizeAggregation
PREWHERE (n <= 5) AND (n >= 2)
WHERE (n <= 5) AND (n >= 2)
)
WHERE (n >= 2) AND (n <= 5)

View File

@ -293,8 +293,8 @@ select * from (select * from tab where (a + b) * c = 8 union all select * from t
select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%';
Prefix sort description: sin(divide(a, b)) ASC
Result sort description: sin(divide(a, b)) ASC
ReadType: InOrder
ReadType: InOrder
ReadType: InOrder
ReadType: InOrder
select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b);
2 2 2 2
2 2 2 2
@ -311,7 +311,7 @@ select * from (select * from tab where (a + b) * c = 8 union all select * from t
select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%';
Prefix sort description: sin(divide(a, b)) ASC
Result sort description: sin(divide(a, b)) ASC
ReadType: InOrder
ReadType: InOrder
ReadType: InOrder
select * from (select * from tab union all select * from tab5) order by (a + b) * c;
0 0 0 0

View File

@ -1,23 +1,15 @@
SELECT i AS i
FROM index_append_test_test
PREWHERE a = 0
WHERE (a = 0) AND indexHint((i + 40) > 0)
SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, optimize_move_to_prewhere = 1, optimize_substitute_columns = 1, optimize_append_index = 1
1
SELECT i AS i
FROM index_append_test_test
PREWHERE a < 0
SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, optimize_move_to_prewhere = 1, optimize_substitute_columns = 1, optimize_append_index = 1
0
SELECT i AS i
FROM index_append_test_test
PREWHERE a >= 0
WHERE (a >= 0) AND indexHint((i + 40) > 0)
SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, optimize_move_to_prewhere = 1, optimize_substitute_columns = 1, optimize_append_index = 1
1
SELECT i AS i
FROM index_append_test_test
PREWHERE (2 * b) < 100
WHERE ((2 * b) < 100) AND indexHint(i < 100)
SETTINGS convert_query_to_cnf = 1, optimize_using_constraints = 1, optimize_move_to_prewhere = 1, optimize_substitute_columns = 1, optimize_append_index = 1
1
Filter column: and(equals(a, 0), indexHint(greater(plus(i, 40), 0))) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: equals(a, 0)
Prewhere info
Prewhere filter
Prewhere filter column: less(a, 0) (removed)
Filter column: and(greaterOrEquals(a, 0), indexHint(greater(plus(i, 40), 0))) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: greaterOrEquals(a, 0)
Filter column: and(less(multiply(2, b), 100), indexHint(less(i, 100))) (removed)
Prewhere info
Prewhere filter
Prewhere filter column: less(multiply(2, b), 100)

View File

@ -1,37 +0,0 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
# We should have correct env vars from shell_config.sh to run this test
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS index_append_test_test;"
$CLICKHOUSE_CLIENT --query "CREATE TABLE index_append_test_test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 ASSUME i <= 2 * b AND i + 40 > a) ENGINE = MergeTree() ORDER BY i;"
$CLICKHOUSE_CLIENT --query "INSERT INTO index_append_test_test VALUES (1, 10, 1), (2, 20, 2);"
function run_with_settings()
{
query="$1 SETTINGS convert_query_to_cnf = 1\
, optimize_using_constraints = 1\
, optimize_move_to_prewhere = 1\
, optimize_substitute_columns = 1\
, optimize_append_index = 1"
if [[ $query =~ "EXPLAIN QUERY TREE" ]]; then query="${query}, allow_experimental_analyzer = 1"; fi
$CLICKHOUSE_CLIENT --query="$query"
}
run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a = 0"
run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a = 0" | grep -Fac "indexHint"
run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a < 0"
run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a < 0" | grep -Fac "indexHint"
run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a >= 0"
run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE a >= 0" | grep -Fac "indexHint"
run_with_settings "EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE 2 * b < 100"
run_with_settings "EXPLAIN QUERY TREE SELECT i FROM index_append_test_test WHERE 2 * b < 100" | grep -Fac "indexHint"
$CLICKHOUSE_CLIENT --query "DROP TABLE index_append_test_test;"

View File

@ -0,0 +1,18 @@
DROP TABLE IF EXISTS index_append_test_test;
CREATE TABLE index_append_test_test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 ASSUME i <= 2 * b AND i + 40 > a) ENGINE = MergeTree() ORDER BY i;
INSERT INTO index_append_test_test VALUES (1, 10, 1), (2, 20, 2);
SET convert_query_to_cnf = 1;
SET optimize_using_constraints = 1;
SET optimize_move_to_prewhere = 1;
SET optimize_substitute_columns = 1;
SET optimize_append_index = 1;
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a = 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a < 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE a >= 0) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT i FROM index_append_test_test WHERE 2 * b < 100) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
DROP TABLE index_append_test_test;

View File

@ -9,7 +9,7 @@ String1_0 String2_0 String3_0 String4_0 1
Expression ((Projection + Before ORDER BY))
Filter (WHERE)
Join (JOIN FillRightFirst)
Filter (( + Before JOIN))
Expression
ReadFromMergeTree (default.t1)
Indexes:
PrimaryKey

View File

@ -1,77 +1,79 @@
ReadFromMergeTree (default.test_index)
Indexes:
MinMax
Keys:
y
Parts: 4/5
Granules: 11/12
Partition
Keys:
y
bitAnd(z, 3)
Parts: 3/4
Granules: 10/11
PrimaryKey
Keys:
x
y
Parts: 2/3
Granules: 6/10
Skip
Name: t_minmax
Description: minmax GRANULARITY 2
Parts: 1/2
Granules: 3/6
Skip
Name: t_set
Description: set GRANULARITY 2
Parts: 1/1
Granules: 2/3
ReadFromMergeTree (default.test_index)
Indexes:
MinMax
Keys:
y
Parts: 4/5
Granules: 11/12
Partition
Keys:
y
bitAnd(z, 3)
Parts: 3/4
Granules: 10/11
PrimaryKey
Keys:
x
y
Parts: 2/3
Granules: 6/10
Skip
Name: t_minmax
Description: minmax GRANULARITY 2
Parts: 1/2
Granules: 3/6
Skip
Name: t_set
Description: set GRANULARITY 2
Parts: 1/1
Granules: 2/3
-----------------
"Node Type": "ReadFromMergeTree",
"Description": "default.test_index",
"Indexes": [
{
"Type": "MinMax",
"Keys": ["y"],
"Initial Parts": 5,
"Selected Parts": 4,
"Initial Granules": 12,
"Selected Granules": 11
},
{
"Type": "Partition",
"Keys": ["y", "bitAnd(z, 3)"],
"Initial Parts": 4,
"Selected Parts": 3,
"Initial Granules": 11,
"Selected Granules": 10
},
{
"Type": "PrimaryKey",
"Keys": ["x", "y"],
"Initial Parts": 3,
"Selected Parts": 2,
"Initial Granules": 10,
"Selected Granules": 6
},
{
"Type": "Skip",
"Name": "t_minmax",
"Description": "minmax GRANULARITY 2",
"Initial Parts": 2,
"Selected Parts": 1,
"Initial Granules": 6,
"Selected Granules": 3
},
{
"Type": "Skip",
"Name": "t_set",
"Description": "set GRANULARITY 2",
"Initial Parts": 1,
"Selected Parts": 1,
"Initial Granules": 3,
"Selected Granules": 2
"Node Type": "ReadFromMergeTree",
"Description": "default.test_index",
"Indexes": [
{
"Type": "MinMax",
"Keys": ["y"],
"Initial Parts": 5,
"Selected Parts": 4,
"Initial Granules": 12,
"Selected Granules": 11
},
{
"Type": "Partition",
"Keys": ["y", "bitAnd(z, 3)"],
"Initial Parts": 4,
"Selected Parts": 3,
"Initial Granules": 11,
"Selected Granules": 10
},
{
"Type": "PrimaryKey",
"Keys": ["x", "y"],
"Initial Parts": 3,
"Selected Parts": 2,
"Initial Granules": 10,
"Selected Granules": 6
},
{
"Type": "Skip",
"Name": "t_minmax",
"Description": "minmax GRANULARITY 2",
"Initial Parts": 2,
"Selected Parts": 1,
"Initial Granules": 6,
"Selected Granules": 3
},
{
"Type": "Skip",
"Name": "t_set",
"Description": "set GRANULARITY 2",
"Initial Parts": 1,
"Selected Parts": 1,
"Initial Granules": 3,
"Selected Granules": 2
}
]
}
]
}
@ -89,15 +91,15 @@
ReadType: InReverseOrder
Parts: 1
Granules: 3
ReadFromMergeTree (default.idx)
Indexes:
PrimaryKey
Keys:
x
plus(x, y)
Condition: or((x in 2-element set), (plus(plus(x, y), 1) in (-Inf, 2]))
Parts: 1/1
Granules: 1/1
ReadFromMergeTree (default.idx)
Indexes:
PrimaryKey
Keys:
x
plus(x, y)
Condition: or((x in 2-element set), (plus(plus(x, y), 1) in (-Inf, 2]))
Parts: 1/1
Granules: 1/1
ReadFromMergeTree (default.test_index)
Indexes:
MinMax

View File

@ -64,8 +64,10 @@ ExpressionTransform
(Sorting)
(Expression)
ExpressionTransform
(ReadFromMergeTree)
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression)
ExpressionTransform
(ReadFromMergeTree)
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression)
ExpressionTransform
(Limit)
@ -91,8 +93,10 @@ ExpressionTransform
PartialSortingTransform
(Expression)
ExpressionTransform
(ReadFromMergeTree)
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression)
ExpressionTransform
(ReadFromMergeTree)
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression)
ExpressionTransform
(Limit)
@ -115,7 +119,7 @@ SELECT
date,
i
FROM t_read_in_order
PREWHERE date = \'2020-10-12\'
WHERE date = \'2020-10-12\'
ORDER BY i DESC
LIMIT 5
(Expression)
@ -125,9 +129,11 @@ ExpressionTransform
(Sorting)
(Expression)
ExpressionTransform
(ReadFromMergeTree)
ReverseTransform
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InReverseOrder) 0 → 1
(Expression)
ExpressionTransform
(ReadFromMergeTree)
ReverseTransform
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InReverseOrder) 0 → 1
(Expression)
ExpressionTransform
(Limit)

View File

@ -56,6 +56,7 @@ algorithm: Thread
Sorting (Stream): a ASC, b ASC
Sorting (Stream): a ASC, b ASC
Sorting (Stream): a ASC, b ASC
Sorting (Stream): a ASC, b ASC
-- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query
-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns
Sorting (Stream): a ASC

View File

@ -1,20 +1,20 @@
1 Hello ClickHouse
2 Hello World
Granules: 6/6
Granules: 2/6
Granules: 6/6
Granules: 2/6
Granules: 6/6
Granules: 2/6
---
1 Hello ClickHouse
2 Hello World
6 World Champion
Granules: 6/6
Granules: 3/6
Granules: 6/6
Granules: 3/6
Granules: 6/6
Granules: 3/6
---
5 OLAP Database
Granules: 6/6
Granules: 1/6
Granules: 6/6
Granules: 1/6
Granules: 6/6
Granules: 1/6

View File

@ -4,7 +4,7 @@
0 1 2 200
Aggregating
Order: a ASC, c ASC
ReadFromMergeTree (default.tab)
ReadFromMergeTree (default.tab)
Aggregating
Order: __table1.a ASC, __table1.c ASC
ReadFromMergeTree (default.tab)

View File

@ -13,7 +13,7 @@ FROM
day_,
type_1
FROM test_grouping_sets_predicate
PREWHERE day_ = \'2023-01-05\'
WHERE day_ = \'2023-01-05\'
GROUP BY
GROUPING SETS (
(day_, type_1),
@ -39,8 +39,10 @@ ExpressionTransform × 2
Copy 1 → 2
(Expression)
ExpressionTransform
(ReadFromMergeTree)
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression)
ExpressionTransform
(ReadFromMergeTree)
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression)
ExpressionTransform × 2
(Filter)
@ -70,8 +72,8 @@ ExpressionTransform × 2
FilterTransform
(Filter)
FilterTransform
(Filter)
FilterTransform
(Expression)
ExpressionTransform
(ReadFromMergeTree)
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression)

View File

@ -1,2 +1,2 @@
ReadFromMergeTree (p1)
Granules: 1/12
Granules: 1/12

View File

@ -1,43 +1,43 @@
1 2 3
1 2 3
1 2 3
ReadFromMergeTree (default.data_02771)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 1/1
Skip
Name: x_idx
Description: minmax GRANULARITY 1
Parts: 0/1
Granules: 0/1
Skip
Name: y_idx
Description: minmax GRANULARITY 1
Parts: 0/0
Granules: 0/0
Skip
Name: xy_idx
Description: minmax GRANULARITY 1
Parts: 0/0
Granules: 0/0
ReadFromMergeTree (default.data_02771)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 1/1
Skip
Name: x_idx
Description: minmax GRANULARITY 1
Parts: 0/1
Granules: 0/1
Skip
Name: y_idx
Description: minmax GRANULARITY 1
Parts: 0/0
Granules: 0/0
ReadFromMergeTree (default.data_02771)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 1/1
Skip
Name: x_idx
Description: minmax GRANULARITY 1
Parts: 0/1
Granules: 0/1
Skip
Name: y_idx
Description: minmax GRANULARITY 1
Parts: 0/0
Granules: 0/0
Skip
Name: xy_idx
Description: minmax GRANULARITY 1
Parts: 0/0
Granules: 0/0
ReadFromMergeTree (default.data_02771)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 1/1
Skip
Name: x_idx
Description: minmax GRANULARITY 1
Parts: 0/1
Granules: 0/1
Skip
Name: y_idx
Description: minmax GRANULARITY 1
Parts: 0/0
Granules: 0/0
ReadFromMergeTree (default.data_02771)
Indexes:
PrimaryKey

View File

@ -1,31 +1,31 @@
CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `b` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After insert
SELECT count()
FROM t1
PREWHERE (a < 10) AND (b < 10)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), less(b, 10)) (removed)
10
0
After drop statistic
SELECT count()
FROM t1
PREWHERE (b < 10) AND (a < 10)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(b, 10), less(a, 10)) (removed)
10
CREATE TABLE default.t1\n(\n `a` Float64,\n `b` Int64,\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After add statistic
CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `b` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After materialize statistic
SELECT count()
FROM t1
PREWHERE (a < 10) AND (b < 10)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), less(b, 10)) (removed)
20
After merge
SELECT count()
FROM t1
PREWHERE (a < 10) AND (b < 10)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), less(b, 10)) (removed)
20
CREATE TABLE default.t1\n(\n `a` Float64 STATISTIC(tdigest),\n `c` Int64 STATISTIC(tdigest),\n `pk` String\n)\nENGINE = MergeTree\nORDER BY pk\nSETTINGS min_bytes_for_wide_part = 0, index_granularity = 8192
After rename
SELECT count()
FROM t1
PREWHERE (a < 10) AND (c < 10)
Prewhere info
Prewhere filter
Prewhere filter column: and(less(a, 10), less(c, 10)) (removed)
20

View File

@ -3,7 +3,7 @@ DROP TABLE IF EXISTS t1;
SET allow_experimental_statistic = 1;
SET allow_statistic_optimize = 1;
CREATE TABLE t1
CREATE TABLE t1
(
a Float64 STATISTIC(tdigest),
b Int64 STATISTIC(tdigest),
@ -16,14 +16,14 @@ SHOW CREATE TABLE t1;
INSERT INTO t1 select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000;
SELECT 'After insert';
EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10;
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT count(*) FROM t1 WHERE b < 10 and a < 10;
SELECT count(*) FROM t1 WHERE b < NULL and a < '10';
ALTER TABLE t1 DROP STATISTIC a, b TYPE tdigest;
SELECT 'After drop statistic';
EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10;
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT count(*) FROM t1 WHERE b < 10 and a < 10;
SHOW CREATE TABLE t1;
@ -38,20 +38,20 @@ ALTER TABLE t1 MATERIALIZE STATISTIC a, b TYPE tdigest;
INSERT INTO t1 select number, -number, generateUUIDv4() FROM system.numbers LIMIT 10000;
SELECT 'After materialize statistic';
EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10;
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT count(*) FROM t1 WHERE b < 10 and a < 10;
OPTIMIZE TABLE t1 FINAL;
SELECT 'After merge';
EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE b < 10 and a < 10;
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE b < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT count(*) FROM t1 WHERE b < 10 and a < 10;
ALTER TABLE t1 RENAME COLUMN b TO c;
SHOW CREATE TABLE t1;
SELECT 'After rename';
EXPLAIN SYNTAX SELECT count(*) FROM t1 WHERE c < 10 and a < 10;
SELECT replaceRegexpAll(explain, '__table1\.|_UInt8', '') FROM (EXPLAIN actions=1 SELECT count(*) FROM t1 WHERE c < 10 and a < 10) WHERE explain LIKE '%Prewhere%' OR explain LIKE '%Filter column%';
SELECT count(*) FROM t1 WHERE c < 10 and a < 10;
DROP TABLE IF EXISTS t1;

View File

@ -1,47 +1,3 @@
CreatingSets
Expression
ReadFromMergeTree
Indexes:
PrimaryKey
Keys:
id
value
Condition: and((id in (-Inf, 10]), (value in 1-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
Expression
ReadFromMergeTree
Indexes:
PrimaryKey
Keys:
id
value
Condition: and((id in (-Inf, 10]), (value in 1-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
Expression
ReadFromMergeTree
Indexes:
PrimaryKey
Keys:
id
value
Condition: and((id in (-Inf, 10]), (value in 5-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
Expression
ReadFromMergeTree
Indexes:
PrimaryKey
Keys:
id
value
Condition: and((id in (-Inf, 10]), (value in 5-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
Expression
Expression
@ -51,7 +7,7 @@ CreatingSets
Keys:
id
value
Condition: and((value in 1-element set), (id in (-Inf, 10]))
Condition: and((id in (-Inf, 10]), (value in 1-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
@ -63,7 +19,7 @@ CreatingSets
Keys:
id
value
Condition: and((value in 1-element set), (id in (-Inf, 10]))
Condition: and((id in (-Inf, 10]), (value in 1-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
@ -75,7 +31,7 @@ CreatingSets
Keys:
id
value
Condition: and((value in 5-element set), (id in (-Inf, 10]))
Condition: and((id in (-Inf, 10]), (value in 5-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
@ -87,6 +43,54 @@ CreatingSets
Keys:
id
value
Condition: and((value in 5-element set), (id in (-Inf, 10]))
Condition: and((id in (-Inf, 10]), (value in 5-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
Expression
Expression
ReadFromMergeTree
Indexes:
PrimaryKey
Keys:
id
value
Condition: and((id in (-Inf, 10]), (value in 1-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
Expression
Expression
ReadFromMergeTree
Indexes:
PrimaryKey
Keys:
id
value
Condition: and((id in (-Inf, 10]), (value in 1-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
Expression
Expression
ReadFromMergeTree
Indexes:
PrimaryKey
Keys:
id
value
Condition: and((id in (-Inf, 10]), (value in 5-element set))
Parts: 1/1
Granules: 1/1
CreatingSets
Expression
Expression
ReadFromMergeTree
Indexes:
PrimaryKey
Keys:
id
value
Condition: and((id in (-Inf, 10]), (value in 5-element set))
Parts: 1/1
Granules: 1/1

View File

@ -7,3 +7,6 @@ Expression ((Projection + Before ORDER BY))
Aggregating
Expression (Before GROUP BY)
ReadFromMerge
ReadFromMergeTree (default.mt1)
ReadFromMergeTree (default.mt2)
ReadFromStorage (TinyLog)

View File

@ -2,12 +2,12 @@
2 Hello World
1 Hello ClickHouse
2 Hello World
Granules: 6/6
Granules: 2/6
Granules: 6/6
Granules: 2/6
Granules: 6/6
Granules: 2/6
Granules: 6/6
Granules: 2/6
Granules: 6/6
Granules: 2/6
Granules: 6/6
Granules: 2/6
---
@ -17,22 +17,22 @@
1 Hello ClickHouse
2 Hello World
6 World Champion
Granules: 6/6
Granules: 3/6
Granules: 6/6
Granules: 3/6
Granules: 6/6
Granules: 3/6
Granules: 6/6
Granules: 3/6
Granules: 6/6
Granules: 3/6
Granules: 6/6
Granules: 3/6
---
5 OLAP Database
5 OLAP Database
Granules: 6/6
Granules: 1/6
Granules: 6/6
Granules: 1/6
Granules: 6/6
Granules: 1/6
Granules: 6/6
Granules: 1/6
Granules: 6/6
Granules: 1/6
Granules: 6/6
Granules: 1/6