mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-02 20:42:04 +00:00
Something working with tests
This commit is contained in:
parent
c23412ae77
commit
9fc0aaf477
@ -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
|
||||
|
@ -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.
|
||||
|
@ -26,6 +26,8 @@ public:
|
||||
|
||||
bool isPreliminary() const { return pre_distinct; }
|
||||
|
||||
UInt64 getLimitHint() const { return limit_hint; }
|
||||
|
||||
private:
|
||||
void updateOutputStream() override;
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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"
|
||||
|
Loading…
Reference in New Issue
Block a user