mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge pull request #37803 from ClickHouse/dictinct_in_order_optimization
DISTINCT in order optimization
This commit is contained in:
commit
2e2ef08712
@ -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.
|
||||
|
||||
|
@ -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");
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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; }
|
||||
|
232
src/Processors/Transforms/DistinctSortedChunkTransform.cpp
Normal file
232
src/Processors/Transforms/DistinctSortedChunkTransform.cpp
Normal 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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
72
src/Processors/Transforms/DistinctSortedChunkTransform.h
Normal file
72
src/Processors/Transforms/DistinctSortedChunkTransform.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
33
tests/performance/distinct_in_order.xml
Normal file
33
tests/performance/distinct_in_order.xml
Normal 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>
|
@ -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
|
@ -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;
|
@ -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
|
43
tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh
Executable file
43
tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.sh
Executable 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"
|
Loading…
Reference in New Issue
Block a user