Something working with tests

This commit is contained in:
Igor Nikonov 2022-09-12 17:01:26 +00:00
parent c23412ae77
commit 9fc0aaf477
7 changed files with 60 additions and 28 deletions

View File

@ -42,7 +42,7 @@ void SortColumnDescription::explain(JSONBuilder::JSONMap & map) const
map.add("With Fill", with_fill);
}
bool SortDescription::hasPrefix(const SortDescription & prefix) const
size_t SortDescription::hasPrefix(const SortDescription & prefix) const
{
if (prefix.empty())
return true;
@ -53,9 +53,9 @@ bool SortDescription::hasPrefix(const SortDescription & prefix) const
for (size_t i = 0; i < prefix.size(); ++i)
{
if ((*this)[i] != prefix[i])
return false;
return i;
}
return true;
return size();
}
#if USE_EMBEDDED_COMPILER

View File

@ -48,6 +48,7 @@ struct SortColumnDescription
bool with_fill;
FillColumnDescription fill_description;
SortColumnDescription() = default;
explicit SortColumnDescription(
const std::string & column_name_,
int direction_ = 1,
@ -120,7 +121,7 @@ public:
size_t min_count_to_compile_sort_description = 3;
bool compile_sort_description = false;
bool hasPrefix(const SortDescription & prefix) const;
size_t hasPrefix(const SortDescription & prefix) const;
};
/** Compile sort description for header_types.

View File

@ -26,6 +26,8 @@ public:
bool isPreliminary() const { return pre_distinct; }
UInt64 getLimitHint() const { return limit_hint; }
private:
void updateOutputStream() override;

View File

@ -1,3 +1,4 @@
#include <memory>
#include <Interpreters/ExpressionActions.h>
#include <Parsers/ASTSelectQuery.h>
#include <Processors/QueryPlan/DistinctStep.h>
@ -6,6 +7,8 @@
#include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Storages/ReadInOrderOptimizer.h>
#include <Common/typeid_cast.h>
#include "Processors/QueryPlan/IQueryPlanStep.h"
#include "Storages/SelectQueryInfo.h"
namespace DB::QueryPlanOptimizations
{
@ -17,13 +20,13 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
/// (1) check if there is preliminary distinct node
/// (2) check if nodes below preliminary distinct preserve sorting
QueryPlan::Node * node = parent_node;
DistinctStep const * pre_distinct = nullptr;
DistinctStep * pre_distinct = nullptr;
while (!node->children.empty())
{
if (pre_distinct)
{
/// check if nodes below DISTINCT preserve sorting
const auto * step = typeid_cast<const ITransformingStep *>(node->step.get());
auto * step = typeid_cast<ITransformingStep *>(node->step.get());
if (step)
{
const ITransformingStep::DataStreamTraits & traits = step->getDataStreamTraits();
@ -31,7 +34,7 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
return 0;
}
}
if (auto const * tmp = typeid_cast<const DistinctStep *>(node->step.get()); tmp)
if (auto * tmp = typeid_cast<DistinctStep *>(node->step.get()); tmp)
{
if (tmp->isPreliminary())
pre_distinct = tmp;
@ -70,7 +73,12 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
if (!distinct_sort_desc.empty())
return 0;
// TODO: provide input order info to read_from_merge_tree
InputOrderInfoPtr order_info
= std::make_shared<const InputOrderInfo>(distinct_sort_desc, distinct_sort_desc.size(), 1, pre_distinct->getLimitHint());
read_from_merge_tree->setQueryInfoInputOrderInfo(order_info);
/// update data stream's sorting properties
pre_distinct->updateInputStream(read_from_merge_tree->getOutputStream());
return 0;
}

View File

@ -149,7 +149,6 @@ ReadFromMergeTree::ReadFromMergeTree(
}
output_stream->sort_description = std::move(sort_description);
}
}
@ -1038,6 +1037,15 @@ void ReadFromMergeTree::setQueryInfoInputOrderInfo(InputOrderInfoPtr order_info)
{
query_info.input_order_info = order_info;
}
/// update sort info for output stream
SortDescription & current_sort_desc = output_stream->sort_description;
size_t prefix_size = current_sort_desc.hasPrefix(order_info->sort_description_for_merging);
if (!prefix_size)
return;
current_sort_desc.resize(prefix_size);
output_stream->sort_mode = DataStream::SortMode::Port;
}
ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const

View File

@ -5,22 +5,22 @@ DistinctTransform
-- enable optimize_distinct_in_order
-- distinct with all primary key columns -> pre-distinct optimization only
DistinctTransform
DistinctSortedChunkTransform
DistinctSortedChunkTransform × 2
-- distinct with primary key prefix -> pre-distinct optimization only
DistinctTransform
DistinctSortedChunkTransform
DistinctSortedChunkTransform × 2
-- distinct with primary key prefix and order by column in distinct -> pre-distinct and final distinct optimization
DistinctSortedTransform
DistinctSortedChunkTransform
DistinctSortedChunkTransform × 2
-- distinct with primary key prefix and order by the same columns -> pre-distinct and final distinct optimization
DistinctSortedStreamTransform
DistinctSortedChunkTransform
DistinctSortedChunkTransform × 2
-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization
DistinctSortedTransform
DistinctSortedChunkTransform
DistinctSortedChunkTransform × 2
-- distinct with primary key prefix and order by column _not_ in distinct -> pre-distinct optimization only
DistinctTransform
DistinctSortedChunkTransform
DistinctSortedChunkTransform × 2
-- distinct with non-primary key prefix -> ordinary distinct
DistinctTransform
DistinctTransform
@ -29,7 +29,11 @@ DistinctSortedTransform
DistinctTransform
-- distinct with non-primary key prefix and order by column _not_ in distinct -> ordinary distinct
DistinctTransform
DistinctTransform
DistinctTransform × 2
-- distinct with non-primary key prefix and order by _const_ column in distinct -> ordinary distinct
DistinctTransform
DistinctTransform
-- Check reading in order for distinct
MergeTreeInOrder × 2 0 → 1
MergeTreeThread × 2 0 → 1
MergeTreeThread × 2 0 → 1

View File

@ -11,44 +11,53 @@ ENABLE_OPTIMIZATION="set optimize_distinct_in_order=1"
GREP_DISTINCT="grep 'DistinctSortedChunkTransform\|DistinctSortedStreamTransform\|DistinctSortedTransform\|DistinctTransform'"
TRIM_LEADING_SPACES="sed -e 's/^[ \t]*//'"
FIND_DISTINCT="$GREP_DISTINCT | $TRIM_LEADING_SPACES"
FIND_READING_IN_ORDER="grep 'MergeTreeInOrder' | $TRIM_LEADING_SPACES"
FIND_READING_DEFAULT="grep 'MergeTreeThread' | $TRIM_LEADING_SPACES"
MAKE_OUTPUT_STABLE="set max_threads=1"
$CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync"
$CLICKHOUSE_CLIENT -q "create table distinct_in_order_explain (a int, b int, c int) engine=MergeTree() order by (a, b)"
$CLICKHOUSE_CLIENT -q "insert into distinct_in_order_explain select number % number, number % 5, number % 10 from numbers(1,10)"
$CLICKHOUSE_CLIENT -q "insert into distinct_in_order_explain select number % number, number % 5, number % 10 from numbers(1,10)"
$CLICKHOUSE_CLIENT -q "select '-- disable optimize_distinct_in_order'"
$CLICKHOUSE_CLIENT -q "select '-- distinct all primary key columns -> ordinary distinct'"
$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "select '-- enable optimize_distinct_in_order'"
$CLICKHOUSE_CLIENT -q "select '-- distinct with all primary key columns -> pre-distinct optimization only'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix -> pre-distinct optimization only'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct -> pre-distinct and final distinct optimization'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by the same columns -> pre-distinct and final distinct optimization'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain order by a, b" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column in distinct but non-primary key prefix -> pre-distinct and final distinct optimization'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b, c from distinct_in_order_explain order by c" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by column _not_ in distinct -> pre-distinct optimization only'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct a, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix -> ordinary distinct'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by column in distinct -> final distinct optimization only'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by b" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by column _not_ in distinct -> ordinary distinct'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by _const_ column in distinct -> ordinary distinct'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, 1 as x from distinct_in_order_explain order by x" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct b, 1 as x from distinct_in_order_explain order by x" | eval $FIND_DISTINCT
$CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync"
echo "-- Check reading in order for distinct"
$CLICKHOUSE_CLIENT -nq "$MAKE_OUTPUT_STABLE;$ENABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_IN_ORDER
$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct a, b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b from distinct_in_order_explain" | eval $FIND_READING_DEFAULT
# $CLICKHOUSE_CLIENT -q "drop table if exists distinct_in_order_explain sync"