Merge pull request #37803 from ClickHouse/dictinct_in_order_optimization

DISTINCT in order optimization
This commit is contained in:
Igor Nikonov 2022-07-03 21:59:04 +02:00 committed by GitHub
commit 2e2ef08712
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 614 additions and 25 deletions

View File

@ -606,6 +606,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Bool, throw_if_no_data_to_insert, true, "Enables or disables empty INSERTs, enabled by default", 0) \
M(Bool, compatibility_ignore_auto_increment_in_create_table, false, "Ignore AUTO_INCREMENT keyword in column declaration if true, otherwise return error. It simplifies migration from MySQL", 0) \
M(Bool, multiple_joins_try_to_keep_original_names, false, "Do not add aliases to top level expression list on multiple joins rewrite", 0) \
M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \
// End of COMMON_SETTINGS
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS.

View File

@ -2572,8 +2572,13 @@ void InterpreterSelectQuery::executeDistinct(QueryPlan & query_plan, bool before
SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode);
auto distinct_step
= std::make_unique<DistinctStep>(query_plan.getCurrentDataStream(), limits, limit_for_distinct, columns, pre_distinct);
auto distinct_step = std::make_unique<DistinctStep>(
query_plan.getCurrentDataStream(),
limits,
limit_for_distinct,
columns,
pre_distinct,
settings.optimize_distinct_in_order);
if (pre_distinct)
distinct_step->setStepDescription("Preliminary DISTINCT");

View File

@ -329,8 +329,13 @@ void InterpreterSelectWithUnionQuery::buildQueryPlan(QueryPlan & query_plan)
/// Add distinct transform
SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode);
auto distinct_step
= std::make_unique<DistinctStep>(query_plan.getCurrentDataStream(), limits, 0, result_header.getNames(), false);
auto distinct_step = std::make_unique<DistinctStep>(
query_plan.getCurrentDataStream(),
limits,
0,
result_header.getNames(),
false,
settings.optimize_distinct_in_order);
query_plan.addStep(std::move(distinct_step));
}

View File

@ -1,8 +1,11 @@
#include <Processors/QueryPlan/DistinctStep.h>
#include <Processors/Transforms/DistinctSortedChunkTransform.h>
#include <Processors/Transforms/DistinctSortedTransform.h>
#include <Processors/Transforms/DistinctTransform.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <IO/Operators.h>
#include <Common/JSONBuilder.h>
#include <Core/SortDescription.h>
namespace DB
{
@ -37,13 +40,26 @@ static ITransformingStep::Traits getTraits(bool pre_distinct, bool already_disti
};
}
static SortDescription getSortDescription(const SortDescription & input_sort_desc, const Names& columns)
{
SortDescription distinct_sort_desc;
for (const auto & sort_column_desc : input_sort_desc)
{
if (std::find(begin(columns), end(columns), sort_column_desc.column_name) == columns.end())
break;
distinct_sort_desc.emplace_back(sort_column_desc);
}
return distinct_sort_desc;
}
DistinctStep::DistinctStep(
const DataStream & input_stream_,
const SizeLimits & set_size_limits_,
UInt64 limit_hint_,
const Names & columns_,
bool pre_distinct_)
bool pre_distinct_,
bool optimize_distinct_in_order_)
: ITransformingStep(
input_stream_,
input_stream_.header,
@ -52,6 +68,7 @@ DistinctStep::DistinctStep(
, limit_hint(limit_hint_)
, columns(columns_)
, pre_distinct(pre_distinct_)
, optimize_distinct_in_order(optimize_distinct_in_order_)
{
if (!output_stream->distinct_columns.empty() /// Columns already distinct, do nothing
&& (!pre_distinct /// Main distinct
@ -65,19 +82,58 @@ DistinctStep::DistinctStep(
void DistinctStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
if (checkColumnsAlreadyDistinct(columns, input_streams.front().distinct_columns))
const auto & input_stream = input_streams.back();
if (checkColumnsAlreadyDistinct(columns, input_stream.distinct_columns))
return;
if (!pre_distinct)
pipeline.resize(1);
pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr
if (optimize_distinct_in_order)
{
if (stream_type != QueryPipelineBuilder::StreamType::Main)
return nullptr;
SortDescription distinct_sort_desc = getSortDescription(input_stream.sort_description, columns);
if (!distinct_sort_desc.empty())
{
/// pre-distinct for sorted chunks
if (pre_distinct)
{
pipeline.addSimpleTransform(
[&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr
{
if (stream_type != QueryPipelineBuilder::StreamType::Main)
return nullptr;
return std::make_shared<DistinctTransform>(header, set_size_limits, limit_hint, columns);
});
return std::make_shared<DistinctSortedChunkTransform>(
header, set_size_limits, limit_hint, distinct_sort_desc, columns);
});
return;
}
/// final distinct for sorted stream (sorting inside and among chunks)
if (input_stream.sort_mode == DataStream::SortMode::Stream)
{
assert(input_stream.has_single_port);
pipeline.addSimpleTransform(
[&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr
{
if (stream_type != QueryPipelineBuilder::StreamType::Main)
return nullptr;
return std::make_shared<DistinctSortedTransform>(header, distinct_sort_desc, set_size_limits, limit_hint, columns);
});
return;
}
}
}
pipeline.addSimpleTransform(
[&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr
{
if (stream_type != QueryPipelineBuilder::StreamType::Main)
return nullptr;
return std::make_shared<DistinctTransform>(header, set_size_limits, limit_hint, columns);
});
}
void DistinctStep::describeActions(FormatSettings & settings) const

View File

@ -10,11 +10,12 @@ class DistinctStep : public ITransformingStep
{
public:
DistinctStep(
const DataStream & input_stream_,
const SizeLimits & set_size_limits_,
UInt64 limit_hint_,
const Names & columns_,
bool pre_distinct_); /// If is enabled, execute distinct for separate streams. Otherwise, merge streams.
const DataStream & input_stream_,
const SizeLimits & set_size_limits_,
UInt64 limit_hint_,
const Names & columns_,
bool pre_distinct_, /// If is enabled, execute distinct for separate streams. Otherwise, merge streams.
bool optimize_distinct_in_order_);
String getName() const override { return "Distinct"; }
@ -30,6 +31,7 @@ private:
UInt64 limit_hint;
Names columns;
bool pre_distinct;
bool optimize_distinct_in_order;
};
}

View File

@ -111,7 +111,7 @@ public:
void describeActions(JSONBuilder::JSONMap & map) const override;
void describeIndexes(JSONBuilder::JSONMap & map) const override;
const StorageID getStorageID() const { return data.getStorageID(); }
StorageID getStorageID() const { return data.getStorageID(); }
UInt64 getSelectedParts() const { return selected_parts; }
UInt64 getSelectedRows() const { return selected_rows; }
UInt64 getSelectedMarks() const { return selected_marks; }

View File

@ -0,0 +1,232 @@
#include <Processors/Transforms/DistinctSortedChunkTransform.h>
namespace DB
{
namespace ErrorCodes
{
extern const int SET_SIZE_LIMIT_EXCEEDED;
}
DistinctSortedChunkTransform::DistinctSortedChunkTransform(
const Block & header_,
const SizeLimits & output_size_limits_,
UInt64 limit_hint_,
const SortDescription & sorted_columns_descr_,
const Names & source_columns)
: ISimpleTransform(header_, header_, true)
, limit_hint(limit_hint_)
, output_size_limits(output_size_limits_)
, sorted_columns_descr(sorted_columns_descr_)
{
/// calculate sorted columns positions
sorted_columns_pos.reserve(sorted_columns_descr.size());
for (auto const & descr : sorted_columns_descr)
{
size_t pos = header_.getPositionByName(descr.column_name);
sorted_columns_pos.emplace_back(pos);
}
/// calculate non-sorted columns positions
other_columns_pos.reserve(source_columns.size());
for (const auto & source_column : source_columns)
{
size_t pos = header_.getPositionByName(source_column);
if (std::find(sorted_columns_pos.begin(), sorted_columns_pos.end(), pos) != sorted_columns_pos.end())
continue;
const auto & col = header_.getByPosition(pos).column;
if (col && !isColumnConst(*col))
other_columns_pos.emplace_back(pos);
}
/// reserve space in auxiliary column vectors for processing
sorted_columns.reserve(sorted_columns_pos.size());
other_columns.reserve(other_columns_pos.size());
current_key.reserve(sorted_columns.size());
}
void DistinctSortedChunkTransform::initChunkProcessing(const Columns & input_columns)
{
sorted_columns.clear();
for (size_t pos : sorted_columns_pos)
sorted_columns.emplace_back(input_columns[pos].get());
other_columns.clear();
for (size_t pos : other_columns_pos)
other_columns.emplace_back(input_columns[pos].get());
if (!other_columns.empty() && data.type == ClearableSetVariants::Type::EMPTY)
data.init(ClearableSetVariants::chooseMethod(other_columns, other_columns_sizes));
}
size_t DistinctSortedChunkTransform::ordinaryDistinctOnRange(IColumn::Filter & filter, size_t range_begin, size_t range_end, bool clear_data)
{
size_t count = 0;
switch (data.type)
{
case ClearableSetVariants::Type::EMPTY:
break;
// clang-format off
#define M(NAME) \
case ClearableSetVariants::Type::NAME: \
count = buildFilterForRange(*data.NAME, filter, range_begin, range_end, clear_data); \
break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
// clang-format on
}
return count;
}
template <typename Method>
size_t DistinctSortedChunkTransform::buildFilterForRange(
Method & method, IColumn::Filter & filter, size_t range_begin, size_t range_end, bool clear_data)
{
typename Method::State state(other_columns, other_columns_sizes, nullptr);
if (clear_data)
method.data.clear();
size_t count = 0;
for (size_t i = range_begin; i < range_end; ++i)
{
auto emplace_result = state.emplaceKey(method.data, i, data.string_pool);
/// emit the record if there is no such key in the current set, skip otherwise
filter[i] = emplace_result.isInserted();
if (filter[i])
++count;
}
return count;
}
void DistinctSortedChunkTransform::setCurrentKey(const size_t row_pos)
{
current_key.clear();
for (auto const & col : sorted_columns)
{
current_key.emplace_back(col->cloneEmpty());
current_key.back()->insertFrom(*col, row_pos);
}
}
bool DistinctSortedChunkTransform::isCurrentKey(const size_t row_pos) const
{
for (size_t i = 0; i < sorted_columns.size(); ++i)
{
int res = current_key[i]->compareAt(0, row_pos, *sorted_columns[i], sorted_columns_descr[i].nulls_direction);
if (res != 0)
return false;
}
return true;
}
size_t DistinctSortedChunkTransform::getRangeEnd(size_t begin, size_t end) const
{
assert(begin < end);
const size_t linear_probe_threadhold = 16;
size_t linear_probe_end = begin + linear_probe_threadhold;
if (linear_probe_end > end)
linear_probe_end = end;
for (size_t pos = begin; pos < linear_probe_end; ++pos)
{
if (!isCurrentKey(pos))
return pos;
}
size_t low = linear_probe_end;
size_t high = end - 1;
while (low <= high)
{
size_t mid = low + (high - low) / 2;
if (isCurrentKey(mid))
low = mid + 1;
else
{
high = mid - 1;
end = mid;
}
}
return end;
}
std::pair<size_t, size_t> DistinctSortedChunkTransform::continueWithPrevRange(const size_t chunk_rows, IColumn::Filter & filter)
{
/// current_key is empty on very first transform() call
/// or first row doesn't match a key from previous transform()
if (current_key.empty() || !isCurrentKey(0))
return {0, 0};
size_t output_rows = 0;
const size_t range_end = getRangeEnd(0, chunk_rows);
if (other_columns.empty())
std::fill(filter.begin(), filter.begin() + range_end, 0); /// skip rows already included in distinct on previous transform()
else
output_rows = ordinaryDistinctOnRange(filter, 0, range_end, false);
return {range_end, output_rows};
}
void DistinctSortedChunkTransform::transform(Chunk & chunk)
{
const size_t chunk_rows = chunk.getNumRows();
if (unlikely(0 == chunk_rows))
return;
Columns input_columns = chunk.detachColumns();
/// split input columns into sorted and other("non-sorted") columns
initChunkProcessing(input_columns);
/// build filter:
/// (1) find range with the same values in sorted columns -> [range_begin, range_end)
/// (2) for found range
/// if there is no "non-sorted" columns: filter out all rows in range except first one
/// otherwise: apply ordinary distinct
/// (3) repeat until chunk is processed
IColumn::Filter filter(chunk_rows);
auto [range_begin, output_rows] = continueWithPrevRange(chunk_rows, filter); /// try to process chuck as continuation of previous one
size_t range_end = range_begin;
while (range_end != chunk_rows)
{
// set current key to find range
setCurrentKey(range_begin);
// find new range [range_begin, range_end)
range_end = getRangeEnd(range_begin, chunk_rows);
// update filter for range
if (other_columns.empty())
{
filter[range_begin] = 1;
std::fill(filter.begin() + range_begin + 1, filter.begin() + range_end, 0);
++output_rows;
}
else
{
// ordinary distinct in range if there are "non-sorted" columns
output_rows += ordinaryDistinctOnRange(filter, range_begin, range_end, true);
}
// set where next range start
range_begin = range_end;
}
/// apply the built filter
for (auto & input_column : input_columns)
input_column = input_column->filter(filter, output_rows);
chunk.setColumns(std::move(input_columns), output_rows);
/// Update total output rows and check limits
total_output_rows += output_rows;
if ((limit_hint && total_output_rows >= limit_hint)
|| !output_size_limits.check(total_output_rows, data.getTotalByteCount(), "DISTINCT", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED))
{
stopReading();
}
}
}

View File

@ -0,0 +1,72 @@
#pragma once
#include <Columns/IColumn.h>
#include <Core/ColumnNumbers.h>
#include <Core/SortDescription.h>
#include <Interpreters/SetVariants.h>
#include <Processors/ISimpleTransform.h>
namespace DB
{
///
/// DISTINCT optimization for sorted chunks
///
/// (1) distinct columns are split into two groups - sorted i.e. belong to sorting prefix,
/// and non-sorted (other columns w/o sorting guarantees).
///
/// (2) Rows are split into ranges. Range is a set of rows where the sorting prefix value is the same.
/// If there are no non-sorted columns, then we just skip all rows in range except one.
/// If there are non-sorted columns, then for each range, we use a hash table to find unique rows in a range.
///
/// (3) The implementation also checks if current chunks is continuation of previous one,
/// i.e. sorting prefix value of last row in previous chunk is the same as of first row in current one,
/// so it can correctly process sorted stream as well.
/// For this, we don't clear sorting prefix value and hash table after a range is processed,
/// only right before a new range processing
///
class DistinctSortedChunkTransform : public ISimpleTransform
{
public:
DistinctSortedChunkTransform(
const Block & header_,
const SizeLimits & output_size_limits_,
UInt64 limit_hint_,
const SortDescription & sorted_columns_descr_,
const Names & source_columns_);
String getName() const override { return "DistinctSortedChunkTransform"; }
protected:
void transform(Chunk & chunk) override;
private:
void initChunkProcessing(const Columns & input_columns);
std::pair<size_t, size_t> continueWithPrevRange(size_t chunk_rows, IColumn::Filter & filter);
size_t ordinaryDistinctOnRange(IColumn::Filter & filter, size_t range_begin, size_t range_end, bool clear_data);
inline void setCurrentKey(size_t row_pos);
inline bool isCurrentKey(size_t row_pos) const;
inline size_t getRangeEnd(size_t range_begin, size_t range_end) const;
template <typename Method>
size_t buildFilterForRange(Method & method, IColumn::Filter & filter, size_t range_begin, size_t range_end, bool clear_data);
ClearableSetVariants data;
const size_t limit_hint;
size_t total_output_rows = 0;
/// Restrictions on the maximum size of the output data.
const SizeLimits output_size_limits;
const SortDescription sorted_columns_descr;
ColumnNumbers sorted_columns_pos;
ColumnRawPtrs sorted_columns; // used during processing
ColumnNumbers other_columns_pos;
Sizes other_columns_sizes;
ColumnRawPtrs other_columns; // used during processing
MutableColumns current_key;
};
}

View File

@ -17,17 +17,13 @@ DistinctTransform::DistinctTransform(
, limit_hint(limit_hint_)
, set_size_limits(set_size_limits_)
{
size_t num_columns = columns_.empty() ? header_.columns() : columns_.size();
key_columns_pos.reserve(columns_.size());
const size_t num_columns = columns_.empty() ? header_.columns() : columns_.size();
key_columns_pos.reserve(num_columns);
for (size_t i = 0; i < num_columns; ++i)
{
auto pos = columns_.empty() ? i
: header_.getPositionByName(columns_[i]);
const auto pos = columns_.empty() ? i : header_.getPositionByName(columns_[i]);
const auto & col = header_.getByPosition(pos).column;
if (!(col && isColumnConst(*col)))
if (col && !isColumnConst(*col))
key_columns_pos.emplace_back(pos);
}
}

View File

@ -0,0 +1,33 @@
<test>
<!-- high cardinality -->
<create_query>CREATE TABLE distinct_cardinality_high (high UInt64, medium UInt64, low UInt64) ENGINE MergeTree() ORDER BY (high, medium)</create_query>
<fill_query>INSERT INTO distinct_cardinality_high SELECT number % 1e6, number % 1e4, number % 1e2 FROM numbers_mt(1e8)</fill_query>
<query>SELECT DISTINCT high FROM distinct_cardinality_high FORMAT Null</query>
<query>SELECT DISTINCT high, low FROM distinct_cardinality_high FORMAT Null</query>
<query>SELECT DISTINCT high, medium FROM distinct_cardinality_high FORMAT Null</query>
<query>SELECT DISTINCT high, medium, low FROM distinct_cardinality_high FORMAT Null</query>
<query>SELECT DISTINCT high, medium FROM distinct_cardinality_high ORDER BY medium FORMAT Null</query>
<query>SELECT DISTINCT high, medium FROM distinct_cardinality_high ORDER BY high FORMAT Null</query>
<query>SELECT DISTINCT high, low FROM distinct_cardinality_high ORDER BY low FORMAT Null</query>
<query>SELECT DISTINCT high, medium, low FROM distinct_cardinality_high ORDER BY low FORMAT Null</query>
<drop_query>DROP TABLE IF EXISTS distinct_cardinality_high</drop_query>
<!-- low cardinality -->
<create_query>CREATE TABLE distinct_cardinality_low (low UInt64, medium UInt64, high UInt64) ENGINE MergeTree() ORDER BY (low, medium)</create_query>
<fill_query>INSERT INTO distinct_cardinality_low SELECT number % 1e2, number % 1e4, number % 1e6 FROM numbers_mt(1e8)</fill_query>
<query>SELECT DISTINCT low FROM distinct_cardinality_low FORMAT Null</query>
<query>SELECT DISTINCT low, medium FROM distinct_cardinality_low FORMAT Null</query>
<query>SELECT DISTINCT low, high FROM distinct_cardinality_low FORMAT Null</query>
<query>SELECT DISTINCT low, medium, high FROM distinct_cardinality_low FORMAT Null</query>
<query>SELECT DISTINCT low, medium FROM distinct_cardinality_low ORDER BY medium FORMAT Null</query>
<query>SELECT DISTINCT low, medium FROM distinct_cardinality_low ORDER BY low FORMAT Null</query>
<query>SELECT DISTINCT low, high FROM distinct_cardinality_low ORDER BY high FORMAT Null</query>
<query>SELECT DISTINCT low, medium, high FROM distinct_cardinality_low ORDER BY high FORMAT Null</query>
<drop_query>DROP TABLE IF EXISTS distinct_cardinality_low</drop_query>
</test>

View File

@ -0,0 +1,79 @@
-- enable distinct in order optimization
-- create table with only primary key columns
-- the same values in every chunk, pre-distinct should skip entire chunks with the same key as previous one
0
-- create table with only primary key columns
-- pre-distinct should skip part of chunk since it contains values from previous one
0
1
2
3
4
5
6
7
8
9
-- create table with not only primary key columns
-- distinct with primary key prefix only
0
-- distinct with primary key prefix only, order by sorted column
0
-- distinct with primary key prefix only, order by sorted column desc
0
-- distinct with full key, order by sorted column
0 0
0 1
0 2
0 3
0 4
-- distinct with full key, order by sorted column desc
0 4
0 3
0 2
0 1
0 0
-- distinct with key prefix and non-sorted column, order by non-sorted
0 0
0 1
0 2
0 3
0 4
0 5
0 6
0 7
0 8
0 9
-- distinct with key prefix and non-sorted column, order by non-sorted desc
0 9
0 8
0 7
0 6
0 5
0 4
0 3
0 2
0 1
0 0
-- distinct with non-key prefix and non-sorted column, order by non-sorted
0 0
1 1
2 2
3 3
4 4
0 5
1 6
2 7
3 8
4 9
-- distinct with non-key prefix and non-sorted column, order by non-sorted desc
4 9
3 8
2 7
1 6
0 5
4 4
3 3
2 2
1 1
0 0

View File

@ -0,0 +1,46 @@
select '-- enable distinct in order optimization';
set optimize_distinct_in_order=1;
select '-- create table with only primary key columns';
drop table if exists distinct_in_order sync;
create table distinct_in_order (a int) engine=MergeTree() order by a settings index_granularity=10;
select '-- the same values in every chunk, pre-distinct should skip entire chunks with the same key as previous one';
insert into distinct_in_order (a) select * from zeros(10);
insert into distinct_in_order (a) select * from zeros(10); -- this entire chunk should be skipped in pre-distinct
select distinct * from distinct_in_order settings max_block_size=10, max_threads=1;
select '-- create table with only primary key columns';
select '-- pre-distinct should skip part of chunk since it contains values from previous one';
drop table if exists distinct_in_order sync;
create table distinct_in_order (a int) engine=MergeTree() order by a settings index_granularity=10;
insert into distinct_in_order (a) select * from zeros(10);
insert into distinct_in_order select * from numbers(10); -- first row (0) from this chunk should be skipped in pre-distinct
select distinct a from distinct_in_order settings max_block_size=10, max_threads=1;
select '-- create table with not only primary key columns';
drop table if exists distinct_in_order sync;
create table distinct_in_order (a int, b int, c int) engine=MergeTree() order by (a, b);
insert into distinct_in_order select number % number, number % 5, number % 10 from numbers(1,1000000);
select '-- distinct with primary key prefix only';
select distinct a from distinct_in_order;
select '-- distinct with primary key prefix only, order by sorted column';
select distinct a from distinct_in_order order by a;
select '-- distinct with primary key prefix only, order by sorted column desc';
select distinct a from distinct_in_order order by a desc;
select '-- distinct with full key, order by sorted column';
select distinct a,b from distinct_in_order order by b;
select '-- distinct with full key, order by sorted column desc';
select distinct a,b from distinct_in_order order by b desc;
select '-- distinct with key prefix and non-sorted column, order by non-sorted';
select distinct a,c from distinct_in_order order by c;
select '-- distinct with key prefix and non-sorted column, order by non-sorted desc';
select distinct a,c from distinct_in_order order by c desc;
select '-- distinct with non-key prefix and non-sorted column, order by non-sorted';
select distinct b,c from distinct_in_order order by c;
select '-- distinct with non-key prefix and non-sorted column, order by non-sorted desc';
select distinct b,c from distinct_in_order order by c desc;
drop table if exists distinct_in_order sync;

View File

@ -0,0 +1,19 @@
-- disable optimize_distinct_in_order
-- distinct all primary key columns -> no optimizations
No optimizations
-- enable optimize_distinct_in_order
-- distinct with all primary key columns -> pre-distinct optimization only
DistinctSortedChunkTransform
-- distinct with primary key prefix -> pre-distinct optimization only
DistinctSortedChunkTransform
-- distinct with primary key prefix and order by on column in distinct -> pre-distinct and final distinct optimization
DistinctSortedTransform
DistinctSortedChunkTransform
-- distinct with primary key prefix and order by on column _not_ in distinct -> pre-distinct optimization only
DistinctSortedChunkTransform
-- distinct with non-primary key prefix -> no optimizations
No optimizations
-- distinct with non-primary key prefix and order by on column in distinct -> final distinct optimization only
DistinctSortedTransform
-- distinct with non-primary key prefix and order by on column _not_ in distinct -> no optimizations
No optimizations

View File

@ -0,0 +1,43 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
[ ! -z "$CLICKHOUSE_CLIENT_REDEFINED" ] && CLICKHOUSE_CLIENT=$CLICKHOUSE_CLIENT_REDEFINED
DISABLE_OPTIMIZATION="set optimize_distinct_in_order=0"
ENABLE_OPTIMIZATION="set optimize_distinct_in_order=1"
GREP_OPTIMIZATIONS="grep 'DistinctSortedChunkTransform\|DistinctSortedTransform'"
TRIM_LEADING_SPACES="sed -e 's/^[ \t]*//'"
FIND_OPTIMIZATIONS="$GREP_OPTIMIZATIONS | $TRIM_LEADING_SPACES"
$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, c)"
$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 -> no optimizations'"
$CLICKHOUSE_CLIENT -nq "$DISABLE_OPTIMIZATION;explain pipeline select distinct * from distinct_in_order_explain" | eval $GREP_OPTIMIZATIONS || echo "No optimizations"
$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_OPTIMIZATIONS
$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_OPTIMIZATIONS
$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by on 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_OPTIMIZATIONS
$CLICKHOUSE_CLIENT -q "select '-- distinct with primary key prefix and order by on 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_OPTIMIZATIONS
$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix -> no optimizations'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain" | eval $GREP_OPTIMIZATIONS || echo "No optimizations"
$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by on 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_OPTIMIZATIONS
$CLICKHOUSE_CLIENT -q "select '-- distinct with non-primary key prefix and order by on column _not_ in distinct -> no optimizations'"
$CLICKHOUSE_CLIENT -nq "$ENABLE_OPTIMIZATION;explain pipeline select distinct b, c from distinct_in_order_explain order by a" | eval $GREP_OPTIMIZATIONS || echo "No optimizations"