Add index to table system.numbers (#50909)

This commit is contained in:
JackyWoo 2023-12-02 02:59:25 +08:00 committed by GitHub
parent 1b49463bd2
commit 8a68d51261
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
53 changed files with 1592 additions and 365 deletions

View File

@ -31,3 +31,26 @@ SELECT * FROM system.numbers LIMIT 10;
10 rows in set. Elapsed: 0.001 sec. 10 rows in set. Elapsed: 0.001 sec.
``` ```
You can also limit the output by predicates.
```sql
SELECT * FROM system.numbers < 10;
```
```response
┌─number─┐
│ 0 │
│ 1 │
│ 2 │
│ 3 │
│ 4 │
│ 5 │
│ 6 │
│ 7 │
│ 8 │
│ 9 │
└────────┘
10 rows in set. Elapsed: 0.001 sec.
```

View File

@ -415,7 +415,7 @@ ExpressionTransform
ExpressionTransform × 2 ExpressionTransform × 2
(SettingQuotaAndLimits) (SettingQuotaAndLimits)
(ReadFromStorage) (ReadFromStorage)
NumbersMt × 2 0 → 1 NumbersRange × 2 0 → 1
``` ```
### EXPLAIN ESTIMATE ### EXPLAIN ESTIMATE

View File

@ -17,6 +17,8 @@ The following queries are equivalent:
SELECT * FROM numbers(10); SELECT * FROM numbers(10);
SELECT * FROM numbers(0, 10); SELECT * FROM numbers(0, 10);
SELECT * FROM system.numbers LIMIT 10; SELECT * FROM system.numbers LIMIT 10;
SELECT * FROM system.numbers WHERE number BETWEEN 0 AND 9;
SELECT * FROM system.numbers WHERE number IN (0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
``` ```
Examples: Examples:

View File

@ -371,7 +371,7 @@ ExpressionTransform
ExpressionTransform × 2 ExpressionTransform × 2
(SettingQuotaAndLimits) (SettingQuotaAndLimits)
(ReadFromStorage) (ReadFromStorage)
NumbersMt × 2 0 → 1 NumbersRange × 2 0 → 1
``` ```
### EXPLAIN ESTIMATE {#explain-estimate} ### EXPLAIN ESTIMATE {#explain-estimate}

157
src/Core/PlainRanges.cpp Normal file
View File

@ -0,0 +1,157 @@
#include <Core/PlainRanges.h>
namespace DB
{
PlainRanges::PlainRanges(const Range & range)
{
ranges.push_back(range);
}
PlainRanges::PlainRanges(const Ranges & ranges_, bool may_have_intersection, bool ordered)
{
if (may_have_intersection)
ranges = ordered ? makePlainFromOrdered(ranges_) : makePlainFromUnordered(ranges_);
else
ranges = ranges_;
}
Ranges PlainRanges::makePlainFromOrdered(const Ranges & ranges_)
{
if (ranges_.size() <= 1)
return ranges_;
Ranges ret{ranges_.front()};
for (size_t i = 1; i < ranges_.size(); ++i)
{
const auto & cur = ranges_[i];
if (ret.back().intersectsRange(cur))
ret.back() = *ret.back().unionWith(cur);
else
ret.push_back(cur);
}
return ret;
}
Ranges PlainRanges::makePlainFromUnordered(Ranges ranges_)
{
if (ranges_.size() <= 1)
return ranges_;
std::sort(ranges_.begin(), ranges_.end(), compareByLeftBound);
return makePlainFromOrdered(ranges_);
}
PlainRanges PlainRanges::unionWith(const PlainRanges & other)
{
auto left_itr = ranges.begin();
auto right_itr = other.ranges.begin();
Ranges new_range;
for (; left_itr != ranges.end() && right_itr != other.ranges.end();)
{
if (left_itr->leftThan(*right_itr))
{
new_range.push_back(*left_itr);
left_itr++;
}
else if (left_itr->rightThan(*right_itr))
{
new_range.push_back(*right_itr);
right_itr++;
}
else /// union
{
new_range.emplace_back(*(left_itr->unionWith(*right_itr)));
if (compareByRightBound(*left_itr, *right_itr))
left_itr++;
else
right_itr++;
}
}
while (left_itr != ranges.end())
{
new_range.push_back(*left_itr);
left_itr++;
}
while (right_itr != other.ranges.end())
{
new_range.push_back(*right_itr);
right_itr++;
}
/// After union two PlainRanges, new ranges may like: [1, 4], [2, 5]
/// We must make them plain.
return PlainRanges(makePlainFromOrdered(new_range));
}
PlainRanges PlainRanges::intersectWith(const PlainRanges & other)
{
auto left_itr = ranges.begin();
auto right_itr = other.ranges.begin();
Ranges new_ranges;
for (; left_itr != ranges.end() && right_itr != other.ranges.end();)
{
if (left_itr->leftThan(*right_itr))
{
left_itr++;
}
else if (left_itr->rightThan(*right_itr))
{
right_itr++;
}
else /// intersection
{
auto intersected = left_itr->intersectWith(*right_itr);
if (intersected) /// skip blank range
new_ranges.emplace_back(*intersected);
if (compareByRightBound(*left_itr, *right_itr))
left_itr++;
else
right_itr++;
}
}
return PlainRanges(new_ranges);
}
bool PlainRanges::compareByLeftBound(const Range & lhs, const Range & rhs)
{
if (lhs.left == NEGATIVE_INFINITY && rhs.left == NEGATIVE_INFINITY)
return false;
return Range::less(lhs.left, rhs.left) || ((!lhs.left_included && rhs.left_included) && Range::equals(lhs.left, rhs.left));
};
bool PlainRanges::compareByRightBound(const Range & lhs, const Range & rhs)
{
if (lhs.right == POSITIVE_INFINITY && rhs.right == POSITIVE_INFINITY)
return false;
return Range::less(lhs.right, rhs.right) || ((!lhs.right_included && rhs.right_included) && Range::equals(lhs.right, rhs.right));
};
std::vector<Ranges> PlainRanges::invert(const Ranges & to_invert_ranges)
{
/// invert a blank ranges
if (to_invert_ranges.empty())
return {makeUniverse().ranges};
std::vector<Ranges> reverted_ranges;
for (const auto & range : to_invert_ranges)
{
if (range.isInfinite())
/// return a blank ranges
return {{}};
reverted_ranges.push_back(range.invertRange());
}
return reverted_ranges;
};
}

46
src/Core/PlainRanges.h Normal file
View File

@ -0,0 +1,46 @@
#pragma once
#include <Core/Range.h>
namespace DB
{
/** A plain ranges is a series of ranges who
* 1. have no intersection in any two of the ranges
* 2. ordered by left side
* 3. does not contain blank range
*
* Example:
* query: (k > 1 and key < 5) or (k > 3 and k < 10) or key in (2, 12)
* original ranges: (1, 5), (3, 10), [2, 2], [12, 12]
* plain ranges: (1, 10), [12, 12]
*
* If it is blank, ranges is empty.
*/
struct PlainRanges
{
Ranges ranges;
explicit PlainRanges(const Range & range);
explicit PlainRanges(const Ranges & ranges_, bool may_have_intersection = false, bool ordered = true);
PlainRanges unionWith(const PlainRanges & other);
PlainRanges intersectWith(const PlainRanges & other);
/// Union ranges and return a new plain(ordered and no intersection) ranges.
/// Example:
/// [1, 3], [2, 4], [6, 8] -> [1, 4], [6, 8]
/// [1, 3], [2, 4], (4, 5] -> [1, 4], [5, 5]
static Ranges makePlainFromUnordered(Ranges ranges_);
static Ranges makePlainFromOrdered(const Ranges & ranges_);
static bool compareByLeftBound(const Range & lhs, const Range & rhs);
static bool compareByRightBound(const Range & lhs, const Range & rhs);
static std::vector<Ranges> invert(const Ranges & to_invert_ranges);
static PlainRanges makeBlank() { return PlainRanges({}); }
static PlainRanges makeUniverse() { return PlainRanges({Range::createWholeUniverseWithoutNull()}); }
};
}

View File

@ -123,6 +123,27 @@ bool Range::leftThan(const FieldRef & x) const
return less(x, right) || (right_included && equals(x, right)); return less(x, right) || (right_included && equals(x, right));
} }
bool Range::rightThan(const Range & x) const
{
return less(x.right, left) || (!(left_included && x.right_included) && equals(left, x.right));
}
bool Range::leftThan(const Range & x) const
{
return less(right, x.left) || (!(x.left_included && right_included) && equals(right, x.left));
}
bool Range::fullBounded() const
{
return left.getType() != Field::Types::Null && right.getType() != Field::Types::Null;
}
/// (-inf, +inf)
bool Range::isInfinite() const
{
return left.isNegativeInfinity() && right.isPositiveInfinity();
}
bool Range::intersectsRange(const Range & r) const bool Range::intersectsRange(const Range & r) const
{ {
/// r to the left of me. /// r to the left of me.
@ -159,6 +180,95 @@ void Range::invert()
std::swap(left_included, right_included); std::swap(left_included, right_included);
} }
Ranges Range::invertRange() const
{
Ranges ranges;
/// For full bounded range will generate two ranges.
if (fullBounded()) /// case: [1, 3] -> (-inf, 1), (3, +inf)
{
ranges.push_back({NEGATIVE_INFINITY, false, left, !left_included});
ranges.push_back({right, !right_included, POSITIVE_INFINITY, false});
}
else if (isInfinite())
{
/// blank ranges
}
else /// case: (-inf, 1] or [1, +inf)
{
Range r = *this;
std::swap(r.left, r.right);
if (r.left.isPositiveInfinity()) /// [1, +inf)
{
r.left = NEGATIVE_INFINITY;
r.right_included = !r.left_included;
r.left_included = false;
}
else if (r.right.isNegativeInfinity()) /// (-inf, 1]
{
r.right = POSITIVE_INFINITY;
r.left_included = !r.right_included;
r.right_included = false;
}
ranges.push_back(r);
}
return ranges;
}
std::optional<Range> Range::intersectWith(const Range & r) const
{
if (!intersectsRange(r))
return {};
bool left_bound_use_mine = true;
bool right_bound_use_mine = true;
if (less(left, r.left) || ((!left_included && r.left_included) && equals(left, r.left)))
left_bound_use_mine = false;
if (less(r.right, right) || ((!r.right_included && right_included) && equals(r.right, right)))
right_bound_use_mine = false;
return Range(
left_bound_use_mine ? left : r.left,
left_bound_use_mine ? left_included : r.left_included,
right_bound_use_mine ? right : r.right,
right_bound_use_mine ? right_included : r.right_included);
}
std::optional<Range> Range::unionWith(const Range & r) const
{
if (!intersectsRange(r) && !nearByWith(r))
return {};
bool left_bound_use_mine = false;
bool right_bound_use_mine = false;
if (less(left, r.left) || ((!left_included && r.left_included) && equals(left, r.left)))
left_bound_use_mine = true;
if (less(r.right, right) || ((!r.right_included && right_included) && equals(r.right, right)))
right_bound_use_mine = true;
return Range(
left_bound_use_mine ? left : r.left,
left_bound_use_mine ? left_included : r.left_included,
right_bound_use_mine ? right : r.right,
right_bound_use_mine ? right_included : r.right_included);
}
bool Range::nearByWith(const Range & r) const
{
/// me locates at left
if (((right_included && !r.left_included) || (!right_included && r.left_included)) && equals(right, r.left))
return true;
/// r locate left
if (((r.right_included && !left_included) || (r.right_included && !left_included)) && equals(r.right, left))
return true;
return false;
}
Range intersect(const Range & a, const Range & b) Range intersect(const Range & a, const Range & b)
{ {
Range res = Range::createWholeUniverse(); Range res = Range::createWholeUniverse();

View File

@ -38,6 +38,13 @@ struct FieldRef : public Field
size_t column_idx = 0; size_t column_idx = 0;
}; };
/** Range with open or closed ends; possibly unbounded.
*/
struct Range;
/** A serious of range who can overlap or non-overlap.
*/
using Ranges = std::vector<Range>;
/** Range with open or closed ends; possibly unbounded. /** Range with open or closed ends; possibly unbounded.
*/ */
struct Range struct Range
@ -79,12 +86,37 @@ public:
/// x is to the right /// x is to the right
bool leftThan(const FieldRef & x) const; bool leftThan(const FieldRef & x) const;
/// completely right than x
bool rightThan(const Range & x) const;
/// completely left than x
bool leftThan(const Range & x) const;
/// range like [1, 2]
bool fullBounded() const;
/// (-inf, +inf)
bool isInfinite() const;
bool isBlank() const;
bool intersectsRange(const Range & r) const; bool intersectsRange(const Range & r) const;
bool containsRange(const Range & r) const; bool containsRange(const Range & r) const;
/// Invert left and right
void invert(); void invert();
/// Invert the range.
/// Example:
/// [1, 3] -> (-inf, 1), (3, +inf)
Ranges invertRange() const;
std::optional<Range> intersectWith(const Range & r) const;
std::optional<Range> unionWith(const Range & r) const;
/// If near by r, they can be combined to a continuous range.
/// TODO If field is integer, case like [2, 3], [4, 5] is excluded.
bool nearByWith(const Range & r) const;
String toString() const; String toString() const;
}; };

View File

@ -105,7 +105,6 @@ namespace DB
{ {
static UInt64 getLimitUIntValue(const ASTPtr & node, const ContextPtr & context, const std::string & expr); static UInt64 getLimitUIntValue(const ASTPtr & node, const ContextPtr & context, const std::string & expr);
static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery & query, const ContextPtr & context);
namespace ErrorCodes namespace ErrorCodes
{ {
@ -1314,19 +1313,19 @@ static UInt64 getLimitUIntValue(const ASTPtr & node, const ContextPtr & context,
} }
static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery & query, const ContextPtr & context) std::pair<UInt64, UInt64> InterpreterSelectQuery::getLimitLengthAndOffset(const ASTSelectQuery & query, const ContextPtr & context_)
{ {
UInt64 length = 0; UInt64 length = 0;
UInt64 offset = 0; UInt64 offset = 0;
if (query.limitLength()) if (query.limitLength())
{ {
length = getLimitUIntValue(query.limitLength(), context, "LIMIT"); length = getLimitUIntValue(query.limitLength(), context_, "LIMIT");
if (query.limitOffset() && length) if (query.limitOffset() && length)
offset = getLimitUIntValue(query.limitOffset(), context, "OFFSET"); offset = getLimitUIntValue(query.limitOffset(), context_, "OFFSET");
} }
else if (query.limitOffset()) else if (query.limitOffset())
offset = getLimitUIntValue(query.limitOffset(), context, "OFFSET"); offset = getLimitUIntValue(query.limitOffset(), context_, "OFFSET");
return {length, offset}; return {length, offset};
} }

View File

@ -134,9 +134,13 @@ public:
static bool isQueryWithFinal(const SelectQueryInfo & info); static bool isQueryWithFinal(const SelectQueryInfo & info);
static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery & query, const ContextPtr & context);
/// Adjust the parallel replicas settings (enabled, disabled) based on the query analysis /// Adjust the parallel replicas settings (enabled, disabled) based on the query analysis
bool adjustParallelReplicasAfterAnalysis(); bool adjustParallelReplicasAfterAnalysis();
private: private:
InterpreterSelectQuery( InterpreterSelectQuery(
const ASTPtr & query_ptr_, const ASTPtr & query_ptr_,

View File

@ -232,12 +232,12 @@ public:
size_t size() const { return ordered_set.at(0)->size(); } size_t size() const { return ordered_set.at(0)->size(); }
const Columns & getOrderedSet() const { return ordered_set; }
bool hasMonotonicFunctionsChain() const; bool hasMonotonicFunctionsChain() const;
BoolMask checkInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types, bool single_point = false) const; BoolMask checkInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types, bool single_point = false) const;
const Columns & getOrderedSet() const { return ordered_set; }
private: private:
// If all arguments in tuple are key columns, we can optimize NOT IN when there is only one element. // If all arguments in tuple are key columns, we can optimize NOT IN when there is only one element.
bool has_all_keys; bool has_all_keys;

View File

@ -67,7 +67,6 @@ public:
constant_node->getValue(), constant_node->getValue(),
constant_node->getResultType(), constant_node->getResultType(),
settings.transform_null_in); settings.transform_null_in);
DataTypes set_element_types = {in_first_argument->getResultType()}; DataTypes set_element_types = {in_first_argument->getResultType()};
const auto * left_tuple_type = typeid_cast<const DataTypeTuple *>(set_element_types.front().get()); const auto * left_tuple_type = typeid_cast<const DataTypeTuple *>(set_element_types.front().get());
if (left_tuple_type && left_tuple_type->getElements().size() != 1) if (left_tuple_type && left_tuple_type->getElements().size() != 1)
@ -90,32 +89,25 @@ public:
return; return;
auto subquery_to_execute = in_second_argument; auto subquery_to_execute = in_second_argument;
if (auto * table_node = in_second_argument->as<TableNode>()) if (auto * table_node = in_second_argument->as<TableNode>())
{ {
auto storage_snapshot = table_node->getStorageSnapshot(); auto storage_snapshot = table_node->getStorageSnapshot();
auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary)); auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary));
size_t columns_to_select_size = columns_to_select.size(); size_t columns_to_select_size = columns_to_select.size();
auto column_nodes_to_select = std::make_shared<ListNode>(); auto column_nodes_to_select = std::make_shared<ListNode>();
column_nodes_to_select->getNodes().reserve(columns_to_select_size); column_nodes_to_select->getNodes().reserve(columns_to_select_size);
NamesAndTypes projection_columns; NamesAndTypes projection_columns;
projection_columns.reserve(columns_to_select_size); projection_columns.reserve(columns_to_select_size);
for (auto & column : columns_to_select) for (auto & column : columns_to_select)
{ {
column_nodes_to_select->getNodes().emplace_back(std::make_shared<ColumnNode>(column, subquery_to_execute)); column_nodes_to_select->getNodes().emplace_back(std::make_shared<ColumnNode>(column, subquery_to_execute));
projection_columns.emplace_back(column.name, column.type); projection_columns.emplace_back(column.name, column.type);
} }
auto subquery_for_table = std::make_shared<QueryNode>(Context::createCopy(planner_context.getQueryContext())); auto subquery_for_table = std::make_shared<QueryNode>(Context::createCopy(planner_context.getQueryContext()));
subquery_for_table->setIsSubquery(true); subquery_for_table->setIsSubquery(true);
subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select); subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select);
subquery_for_table->getJoinTree() = std::move(subquery_to_execute); subquery_for_table->getJoinTree() = std::move(subquery_to_execute);
subquery_for_table->resolveProjectionColumns(std::move(projection_columns)); subquery_for_table->resolveProjectionColumns(std::move(projection_columns));
subquery_to_execute = std::move(subquery_for_table); subquery_to_execute = std::move(subquery_for_table);
} }

View File

@ -135,7 +135,6 @@ public:
static SetKey createSetKey(const DataTypePtr & left_operand_type, const QueryTreeNodePtr & set_source_node); static SetKey createSetKey(const DataTypePtr & left_operand_type, const QueryTreeNodePtr & set_source_node);
PreparedSets & getPreparedSets() { return prepared_sets; } PreparedSets & getPreparedSets() { return prepared_sets; }
private: private:
/// Query context /// Query context
ContextMutablePtr query_context; ContextMutablePtr query_context;

View File

@ -0,0 +1,529 @@
#include <Processors/QueryPlan/ReadFromSystemNumbersStep.h>
#include <Core/ColumnWithTypeAndName.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Processors/LimitTransform.h>
#include <Processors/Sources/NullSource.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Storages/MergeTree/KeyCondition.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_MANY_ROWS;
}
namespace
{
class NumbersSource : public ISource
{
public:
NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 step_)
: ISource(createHeader()), block_size(block_size_), next(offset_), step(step_)
{
}
String getName() const override { return "Numbers"; }
static Block createHeader() { return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "number")}; }
protected:
Chunk generate() override
{
auto column = ColumnUInt64::create(block_size);
ColumnUInt64::Container & vec = column->getData();
size_t curr = next; /// The local variable for some reason works faster (>20%) than member of class.
UInt64 * pos = vec.data(); /// This also accelerates the code.
UInt64 * end = &vec[block_size];
while (pos < end)
*pos++ = curr++;
next += step;
progress(column->size(), column->byteSize());
return {Columns{std::move(column)}, block_size};
}
private:
UInt64 block_size;
UInt64 next;
UInt64 step;
};
UInt128 sizeOfRange(const Range & r)
{
UInt128 size;
if (r.right.isPositiveInfinity())
return static_cast<UInt128>(std::numeric_limits<uint64_t>::max()) - r.left.get<UInt64>() + r.left_included;
size = static_cast<UInt128>(r.right.get<UInt64>()) - r.left.get<UInt64>() + 1;
if (!r.left_included)
size--;
if (!r.right_included)
size--;
assert(size >= 0);
return size;
};
auto sizeOfRanges(const Ranges & rs)
{
UInt128 total_size{};
for (const Range & r : rs)
{
/// total_size will never overflow
total_size += sizeOfRange(r);
}
return total_size;
};
/// Generate numbers according to ranges.
/// Numbers generated is ordered in one stream.
/// Notice that we will not generate additional numbers out of ranges.
class NumbersRangedSource : public ISource
{
public:
/// Represent a position in Ranges list.
struct RangesPos
{
size_t offset_in_ranges;
UInt128 offset_in_range;
};
struct RangesState
{
RangesPos pos;
mutable std::mutex mutex;
};
using RangesStatePtr = std::shared_ptr<RangesState>;
NumbersRangedSource(const Ranges & ranges_, RangesStatePtr & ranges_state_, UInt64 base_block_size_)
: ISource(NumbersSource::createHeader()), ranges(ranges_), ranges_state(ranges_state_), base_block_size(base_block_size_)
{
}
String getName() const override { return "NumbersRange"; }
protected:
/// Find the data range in ranges and return how many item found.
/// If no data left in ranges return 0.
UInt64 findRanges(RangesPos & start, RangesPos & end, UInt64 base_block_size_)
{
std::lock_guard lock(ranges_state->mutex);
UInt64 need = base_block_size_;
UInt64 size = 0; /// how many item found.
/// find start
start = ranges_state->pos;
end = start;
/// find end
while (need != 0)
{
UInt128 can_provide = end.offset_in_ranges == ranges.size() ? static_cast<UInt128>(0)
: sizeOfRange(ranges[end.offset_in_ranges]) - end.offset_in_range;
if (can_provide == 0)
break;
if (can_provide > need)
{
end.offset_in_range += need;
size += need;
need = 0;
}
else if (can_provide == need)
{
end.offset_in_ranges++;
end.offset_in_range = 0;
size += need;
need = 0;
}
else
{
end.offset_in_ranges++;
end.offset_in_range = 0;
size += static_cast<UInt64>(can_provide);
need -= static_cast<UInt64>(can_provide);
}
}
ranges_state->pos = end;
return size;
}
Chunk generate() override
{
if (ranges.empty())
return {};
auto first_value = [](const Range & r) { return r.left.get<UInt64>() + (r.left_included ? 0 : 1); };
auto last_value = [](const Range & r) { return r.right.get<UInt64>() - (r.right_included ? 0 : 1); };
/// Find the data range.
/// If data left is small, shrink block size.
RangesPos start, end;
auto block_size = findRanges(start, end, base_block_size);
if (!block_size)
return {};
auto column = ColumnUInt64::create(block_size);
ColumnUInt64::Container & vec = column->getData();
/// This will accelerates the code.
UInt64 * pos = vec.data();
UInt64 provided = 0;
RangesPos cursor = start;
while (block_size - provided != 0)
{
UInt64 need = block_size - provided;
auto & range = ranges[cursor.offset_in_ranges];
UInt128 can_provide = cursor.offset_in_ranges == end.offset_in_ranges
? end.offset_in_range - cursor.offset_in_range
: static_cast<UInt128>(last_value(range)) - first_value(range) + 1 - cursor.offset_in_range;
/// set value to block
auto set_value = [&pos](UInt128 & start_value, UInt128 & end_value)
{
if (end_value > std::numeric_limits<UInt64>::max())
{
while (start_value < end_value)
*(pos++) = start_value++;
}
else
{
auto start_value_64 = static_cast<UInt64>(start_value);
auto end_value_64 = static_cast<UInt64>(end_value);
while (start_value_64 < end_value_64)
*(pos++) = start_value_64++;
}
};
if (can_provide > need)
{
UInt64 start_value = first_value(range) + cursor.offset_in_range;
UInt64 end_value = start_value + need; /// end_value will never overflow
while (start_value < end_value)
*(pos++) = start_value++;
provided += need;
cursor.offset_in_range += need;
}
else if (can_provide == need)
{
/// to avoid UInt64 overflow
UInt128 start_value = static_cast<UInt128>(first_value(range)) + cursor.offset_in_range;
UInt128 end_value = start_value + need;
set_value(start_value, end_value);
provided += need;
cursor.offset_in_ranges++;
cursor.offset_in_range = 0;
}
else
{
/// to avoid UInt64 overflow
UInt128 start_value = static_cast<UInt128>(first_value(range)) + cursor.offset_in_range;
UInt128 end_value = start_value + can_provide;
set_value(start_value, end_value);
provided += static_cast<UInt64>(can_provide);
cursor.offset_in_ranges++;
cursor.offset_in_range = 0;
}
}
chassert(block_size == UInt64(pos - vec.begin()));
progress(column->size(), column->byteSize());
return {Columns{std::move(column)}, block_size};
}
private:
/// The ranges is shared between all streams.
Ranges ranges;
/// Ranges state shared between all streams, actually is the start of the ranges.
RangesStatePtr ranges_state;
/// Base block size, will shrink when data left is not enough.
UInt64 base_block_size;
};
}
namespace
{
/// Whether we should push limit down to scan.
bool shouldPushdownLimit(SelectQueryInfo & query_info, UInt64 limit_length)
{
const auto & query = query_info.query->as<ASTSelectQuery &>();
/// Just ignore some minor cases, such as:
/// select * from system.numbers order by number asc limit 10
return !query.distinct && !query.limitBy() && !query_info.has_order_by
&& !query_info.need_aggregate
/// For new analyzer, window will be delete from AST, so we should not use query.window()
&& !query_info.has_window && !query_info.additional_filter_ast && (limit_length > 0 && !query.limit_with_ties);
}
/// Shrink ranges to size.
/// For example: ranges: [1, 5], [8, 100]; size: 7, we will get [1, 5], [8, 9]
void shrinkRanges(Ranges & ranges, size_t size)
{
size_t last_range_idx = 0;
for (size_t i = 0; i < ranges.size(); i++)
{
auto range_size = sizeOfRange(ranges[i]);
if (range_size < size)
{
size -= static_cast<UInt64>(range_size);
continue;
}
else if (range_size == size)
{
last_range_idx = i;
break;
}
else
{
auto & range = ranges[i];
UInt64 right = range.left.get<UInt64>() + static_cast<UInt64>(size);
range.right = Field(right);
range.right_included = !range.left_included;
last_range_idx = i;
break;
}
}
/// delete the additional ranges
ranges.erase(ranges.begin() + (last_range_idx + 1), ranges.end());
}
}
ReadFromSystemNumbersStep::ReadFromSystemNumbersStep(
const Names & column_names_,
StoragePtr storage_,
const StorageSnapshotPtr & storage_snapshot_,
SelectQueryInfo & query_info,
ContextPtr context_,
size_t max_block_size_,
size_t num_streams_)
: SourceStepWithFilter{DataStream{.header = storage_snapshot_->getSampleBlockForColumns(column_names_)}}
, column_names{column_names_}
, storage{std::move(storage_)}
, storage_snapshot{storage_snapshot_}
, context{std::move(context_)}
, key_expression{KeyDescription::parse(column_names[0], storage_snapshot->getMetadataForQuery()->columns, context).expression}
, max_block_size{max_block_size_}
, num_streams{num_streams_}
, limit_length_and_offset(InterpreterSelectQuery::getLimitLengthAndOffset(query_info.query->as<ASTSelectQuery&>(), context))
, should_pushdown_limit(shouldPushdownLimit(query_info, limit_length_and_offset.first))
, limit(query_info.limit)
, storage_limits(query_info.storage_limits)
{
storage_snapshot->check(column_names);
chassert(column_names.size() == 1);
chassert(storage->as<StorageSystemNumbers>() != nullptr);
}
void ReadFromSystemNumbersStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
auto pipe = makePipe();
if (pipe.empty())
{
assert(output_stream != std::nullopt);
pipe = Pipe(std::make_shared<NullSource>(output_stream->header));
}
/// Add storage limits.
for (const auto & processor : pipe.getProcessors())
processor->setStorageLimits(storage_limits);
/// Add to processors to get processor info through explain pipeline statement.
for (const auto & processor : pipe.getProcessors())
processors.emplace_back(processor);
pipeline.init(std::move(pipe));
}
Pipe ReadFromSystemNumbersStep::makePipe()
{
auto & numbers_storage = storage->as<StorageSystemNumbers &>();
if (!numbers_storage.multithreaded)
num_streams = 1;
/// Build rpn of query filters
KeyCondition condition(buildFilterDAG(), context, column_names, key_expression, NameSet{});
Pipe pipe;
Ranges ranges;
if (condition.extractPlainRanges(ranges))
{
/// Intersect ranges with table range
std::optional<Range> table_range;
std::optional<Range> overflowed_table_range;
if (numbers_storage.limit.has_value())
{
if (std::numeric_limits<UInt64>::max() - numbers_storage.offset >= *(numbers_storage.limit))
{
table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(numbers_storage.offset + *(numbers_storage.limit)), false);
}
/// UInt64 overflow, for example: SELECT number FROM numbers(18446744073709551614, 5)
else
{
table_range.emplace(FieldRef(numbers_storage.offset), true, std::numeric_limits<UInt64>::max(), true);
auto overflow_end = UInt128(numbers_storage.offset) + UInt128(*numbers_storage.limit);
overflowed_table_range.emplace(
FieldRef(UInt64(0)), true, FieldRef(UInt64(overflow_end - std::numeric_limits<UInt64>::max() - 1)), false);
}
}
else
{
table_range.emplace(FieldRef(numbers_storage.offset), true, FieldRef(std::numeric_limits<UInt64>::max()), true);
}
Ranges intersected_ranges;
for (auto & r : ranges)
{
auto intersected_range = table_range->intersectWith(r);
if (intersected_range)
intersected_ranges.push_back(*intersected_range);
}
/// intersection with overflowed_table_range goes back.
if (overflowed_table_range.has_value())
{
for (auto & r : ranges)
{
auto intersected_range = overflowed_table_range->intersectWith(r);
if (intersected_range)
intersected_ranges.push_back(*overflowed_table_range);
}
}
/// ranges is blank, return a source who has no data
if (intersected_ranges.empty())
{
pipe.addSource(std::make_shared<NullSource>(NumbersSource::createHeader()));
return pipe;
}
const auto & limit_length = limit_length_and_offset.first;
const auto & limit_offset = limit_length_and_offset.second;
/// If intersected ranges is limited or we can pushdown limit.
if (!intersected_ranges.rbegin()->right.isPositiveInfinity() || should_pushdown_limit)
{
UInt128 total_size = sizeOfRanges(intersected_ranges);
UInt128 query_limit = limit_length + limit_offset;
/// limit total_size by query_limit
if (should_pushdown_limit && query_limit < total_size)
{
total_size = query_limit;
/// We should shrink intersected_ranges for case:
/// intersected_ranges: [1, 4], [7, 100]; query_limit: 2
shrinkRanges(intersected_ranges, total_size);
}
checkLimits(size_t(total_size));
if (total_size / max_block_size < num_streams)
num_streams = static_cast<size_t>(total_size / max_block_size);
if (num_streams == 0)
num_streams = 1;
/// Ranges state, all streams will share the state.
auto ranges_state = std::make_shared<NumbersRangedSource::RangesState>();
for (size_t i = 0; i < num_streams; ++i)
{
auto source = std::make_shared<NumbersRangedSource>(intersected_ranges, ranges_state, max_block_size);
if (i == 0)
source->addTotalRowsApprox(total_size);
pipe.addSource(std::move(source));
}
return pipe;
}
}
/// Fall back to NumbersSource
for (size_t i = 0; i < num_streams; ++i)
{
auto source
= std::make_shared<NumbersSource>(max_block_size, numbers_storage.offset + i * max_block_size, num_streams * max_block_size);
if (numbers_storage.limit && i == 0)
{
auto rows_appr = *(numbers_storage.limit);
if (limit > 0 && limit < rows_appr)
rows_appr = limit;
source->addTotalRowsApprox(rows_appr);
}
pipe.addSource(std::move(source));
}
if (numbers_storage.limit)
{
size_t i = 0;
auto storage_limit = *(numbers_storage.limit);
/// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly.
pipe.addSimpleTransform(
[&](const Block & header)
{
++i;
return std::make_shared<LimitTransform>(header, storage_limit * i / num_streams - storage_limit * (i - 1) / num_streams, 0);
});
}
return pipe;
}
ActionsDAGPtr ReadFromSystemNumbersStep::buildFilterDAG()
{
std::unordered_map<std::string, ColumnWithTypeAndName> node_name_to_input_node_column;
return ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, node_name_to_input_node_column, context);
}
void ReadFromSystemNumbersStep::checkLimits(size_t rows)
{
const auto & settings = context->getSettingsRef();
if (settings.read_overflow_mode == OverflowMode::THROW && settings.max_rows_to_read)
{
const auto limits = SizeLimits(settings.max_rows_to_read, 0, settings.read_overflow_mode);
limits.check(rows, 0, "rows (controlled by 'max_rows_to_read' setting)", ErrorCodes::TOO_MANY_ROWS);
}
if (settings.read_overflow_mode_leaf == OverflowMode::THROW && settings.max_rows_to_read_leaf)
{
const auto leaf_limits = SizeLimits(settings.max_rows_to_read_leaf, 0, settings.read_overflow_mode_leaf);
leaf_limits.check(rows, 0, "rows (controlled by 'max_rows_to_read_leaf' setting)", ErrorCodes::TOO_MANY_ROWS);
}
}
}

View File

@ -0,0 +1,49 @@
#pragma once
#include <Core/QueryProcessingStage.h>
#include <Interpreters/Context_fwd.h>
#include <Processors/QueryPlan/SourceStepWithFilter.h>
#include <QueryPipeline/Pipe.h>
#include <Storages/IStorage_fwd.h>
#include <Storages/SelectQueryInfo.h>
#include <Storages/StorageSnapshot.h>
namespace DB
{
class ReadFromSystemNumbersStep final : public SourceStepWithFilter
{
public:
ReadFromSystemNumbersStep(
const Names & column_names_,
StoragePtr storage_,
const StorageSnapshotPtr & storage_snapshot_,
SelectQueryInfo & query_info,
ContextPtr context_,
size_t max_block_size_,
size_t num_streams_);
String getName() const override { return "ReadFromSystemNumbers"; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
private:
/// Fail fast if estimated number of rows to read exceeds the limit
void checkLimits(size_t rows);
Pipe makePipe();
ActionsDAGPtr buildFilterDAG();
const Names column_names;
StoragePtr storage;
StorageSnapshotPtr storage_snapshot;
ContextPtr context;
ExpressionActionsPtr key_expression;
size_t max_block_size;
size_t num_streams;
std::pair<UInt64, UInt64> limit_length_and_offset;
bool should_pushdown_limit;
UInt64 limit;
std::shared_ptr<const StorageLimitsList> storage_limits;
};
}

View File

@ -46,7 +46,6 @@ namespace ErrorCodes
extern const int BAD_TYPE_OF_FIELD; extern const int BAD_TYPE_OF_FIELD;
} }
/// Returns the prefix of like_pattern before the first wildcard, e.g. 'Hello\_World% ...' --> 'Hello\_World' /// Returns the prefix of like_pattern before the first wildcard, e.g. 'Hello\_World% ...' --> 'Hello\_World'
/// We call a pattern "perfect prefix" if: /// We call a pattern "perfect prefix" if:
/// - (1) the pattern has a wildcard /// - (1) the pattern has a wildcard
@ -799,10 +798,13 @@ KeyCondition::KeyCondition(
if (!filter_node) if (!filter_node)
{ {
has_filter = false;
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
return; return;
} }
has_filter = true;
/** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)), /** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)),
* the use of NOT operator in predicate will result in the indexing algorithm leave out some data. * the use of NOT operator in predicate will result in the indexing algorithm leave out some data.
* This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict * This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict
@ -875,10 +877,13 @@ KeyCondition::KeyCondition(
if (!filter_dag) if (!filter_dag)
{ {
has_filter = false;
rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN);
return; return;
} }
has_filter = true;
auto inverted_dag = cloneASTWithInversionPushDown({filter_dag->getOutputs().at(0)}, context); auto inverted_dag = cloneASTWithInversionPushDown({filter_dag->getOutputs().at(0)}, context);
assert(inverted_dag->getOutputs().size() == 1); assert(inverted_dag->getOutputs().size() == 1);
@ -2540,6 +2545,173 @@ bool KeyCondition::matchesExactContinuousRange() const
return true; return true;
} }
bool KeyCondition::extractPlainRanges(Ranges & ranges) const
{
if (key_indices.empty() || key_indices.size() > 1)
return false;
if (hasMonotonicFunctionsChain())
return false;
/// All Ranges in rpn_stack is plain.
std::stack<PlainRanges> rpn_stack;
for (const auto & element : rpn)
{
if (element.function == RPNElement::FUNCTION_AND)
{
auto right_ranges = rpn_stack.top();
rpn_stack.pop();
auto left_ranges = rpn_stack.top();
rpn_stack.pop();
auto new_range = left_ranges.intersectWith(right_ranges);
rpn_stack.emplace(std::move(new_range));
}
else if (element.function == RPNElement::FUNCTION_OR)
{
auto right_ranges = rpn_stack.top();
rpn_stack.pop();
auto left_ranges = rpn_stack.top();
rpn_stack.pop();
auto new_range = left_ranges.unionWith(right_ranges);
rpn_stack.emplace(std::move(new_range));
}
else if (element.function == RPNElement::FUNCTION_NOT)
{
auto to_invert_ranges = rpn_stack.top();
rpn_stack.pop();
std::vector<Ranges> reverted_ranges = PlainRanges::invert(to_invert_ranges.ranges);
if (reverted_ranges.size() == 1)
rpn_stack.emplace(std::move(reverted_ranges[0]));
else
{
/// intersect reverted ranges
PlainRanges intersected_ranges(reverted_ranges[0]);
for (size_t i = 1; i < reverted_ranges.size(); i++)
{
intersected_ranges = intersected_ranges.intersectWith(PlainRanges(reverted_ranges[i]));
}
rpn_stack.emplace(std::move(intersected_ranges));
}
}
else /// atom relational expression or constants
{
if (element.function == RPNElement::FUNCTION_IN_RANGE)
{
rpn_stack.push(PlainRanges(element.range));
}
else if (element.function == RPNElement::FUNCTION_NOT_IN_RANGE)
{
rpn_stack.push(PlainRanges(element.range.invertRange()));
}
else if (element.function == RPNElement::FUNCTION_IN_SET)
{
if (element.set_index->hasMonotonicFunctionsChain())
return false;
if (element.set_index->size() == 0)
{
rpn_stack.push(PlainRanges::makeBlank()); /// skip blank range
continue;
}
const auto & values = element.set_index->getOrderedSet();
Ranges points_range;
/// values in set_index are ordered and no duplication
for (size_t i=0; i<element.set_index->size(); i++)
{
FieldRef f;
values[0]->get(i, f);
if (f.isNull())
return false;
points_range.push_back({f});
}
rpn_stack.push(PlainRanges(points_range));
}
else if (element.function == RPNElement::FUNCTION_NOT_IN_SET)
{
if (element.set_index->hasMonotonicFunctionsChain())
return false;
if (element.set_index->size() == 0)
{
rpn_stack.push(PlainRanges::makeUniverse());
continue;
}
const auto & values = element.set_index->getOrderedSet();
Ranges points_range;
std::optional<FieldRef> pre;
for (size_t i=0; i<element.set_index->size(); i++)
{
FieldRef cur;
values[0]->get(i, cur);
if (cur.isNull())
return false;
if (pre)
{
Range r(*pre, false, cur, false);
/// skip blank range
if (!(r.left > r.right || (r.left == r.right && !r.left_included && !r.right_included)))
points_range.push_back(r);
}
else
{
points_range.push_back(Range::createRightBounded(cur, false));
}
pre = cur;
}
points_range.push_back(Range::createLeftBounded(*pre, false));
rpn_stack.push(PlainRanges(points_range));
}
else if (element.function == RPNElement::ALWAYS_FALSE)
{
/// skip blank range
rpn_stack.push(PlainRanges::makeBlank());
}
else if (element.function == RPNElement::ALWAYS_TRUE)
{
rpn_stack.push(PlainRanges::makeUniverse());
}
else if (element.function == RPNElement::FUNCTION_IS_NULL)
{
/// key values can not be null, so isNull will get blank range.
rpn_stack.push(PlainRanges::makeBlank());
}
else if (element.function == RPNElement::FUNCTION_IS_NOT_NULL)
{
rpn_stack.push(PlainRanges::makeUniverse());
}
else /// FUNCTION_UNKNOWN
{
if (!has_filter)
rpn_stack.push(PlainRanges::makeUniverse());
else
return false;
}
}
}
if (rpn_stack.size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected stack size in KeyCondition::extractPlainRanges");
for (auto & r : rpn_stack.top().ranges)
{
ranges.push_back(std::move(r));
}
return true;
}
BoolMask KeyCondition::checkInHyperrectangle( BoolMask KeyCondition::checkInHyperrectangle(
const Hyperrectangle & hyperrectangle, const Hyperrectangle & hyperrectangle,
const DataTypes & data_types) const const DataTypes & data_types) const

View File

@ -4,6 +4,7 @@
#include <Core/SortDescription.h> #include <Core/SortDescription.h>
#include <Core/Range.h> #include <Core/Range.h>
#include <Core/PlainRanges.h>
#include <Parsers/ASTExpressionList.h> #include <Parsers/ASTExpressionList.h>
@ -162,6 +163,16 @@ public:
bool matchesExactContinuousRange() const; bool matchesExactContinuousRange() const;
/// Extract plain ranges of the condition.
/// Note that only support one column key condition.
///
/// Now some cases are parsed to unknown function:
/// 1. where 1=1
/// 2. where true
/// 3. no where
/// TODO handle the cases when generate RPN.
bool extractPlainRanges(Ranges & ranges) const;
/// The expression is stored as Reverse Polish Notation. /// The expression is stored as Reverse Polish Notation.
struct RPNElement struct RPNElement
{ {
@ -326,6 +337,10 @@ private:
RPN rpn; RPN rpn;
/// If query has no filter, rpn will has one element with unknown function.
/// This flag identify whether there are filters.
bool has_filter;
ColumnIndices key_columns; ColumnIndices key_columns;
std::vector<size_t> key_indices; std::vector<size_t> key_indices;

View File

@ -20,6 +20,7 @@
#include <Processors/Sources/SourceFromSingleChunk.h> #include <Processors/Sources/SourceFromSingleChunk.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h> #include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Storages/StorageFactory.h> #include <Storages/StorageFactory.h>
#include <Storages/checkAndGetLiteralArgument.h> #include <Storages/checkAndGetLiteralArgument.h>
@ -145,8 +146,11 @@ void StorageExecutable::read(
for (auto & input_query : input_queries) for (auto & input_query : input_queries)
{ {
InterpreterSelectWithUnionQuery interpreter(input_query, context, {}); QueryPipelineBuilder builder;
auto builder = interpreter.buildQueryPipeline(); if (context->getSettings().allow_experimental_analyzer)
builder = InterpreterSelectQueryAnalyzer(input_query, context, {}).buildQueryPipeline();
else
builder = InterpreterSelectWithUnionQuery(input_query, context, {}).buildQueryPipeline();
inputs.emplace_back(QueryPipelineBuilder::getPipe(std::move(builder), resources)); inputs.emplace_back(QueryPipelineBuilder::getPipe(std::move(builder), resources));
} }

View File

@ -1,201 +1,40 @@
#include <Common/Exception.h> #include <Storages/System/StorageSystemNumbers.h>
#include <mutex>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Storages/System/StorageSystemNumbers.h> #include <Interpreters/InterpreterSelectQuery.h>
#include <Storages/SelectQueryInfo.h>
#include <Processors/ISource.h> #include <Processors/ISource.h>
#include <QueryPipeline/Pipe.h>
#include <Processors/LimitTransform.h> #include <Processors/LimitTransform.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/ReadFromSystemNumbersStep.h>
#include <Processors/Sources/NullSource.h>
#include <QueryPipeline/Pipe.h>
#include <Storages/SelectQueryInfo.h>
namespace DB namespace DB
{ {
namespace StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, std::optional<UInt64> limit_, UInt64 offset_)
{ : IStorage(table_id), multithreaded(multithreaded_), limit(limit_), offset(offset_)
class NumbersSource : public ISource
{
public:
NumbersSource(UInt64 block_size_, UInt64 offset_, UInt64 step_)
: ISource(createHeader()), block_size(block_size_), next(offset_), step(step_) {}
String getName() const override { return "Numbers"; }
protected:
Chunk generate() override
{
auto column = ColumnUInt64::create(block_size);
ColumnUInt64::Container & vec = column->getData();
size_t curr = next; /// The local variable for some reason works faster (>20%) than member of class.
UInt64 * pos = vec.data(); /// This also accelerates the code.
UInt64 * end = &vec[block_size];
while (pos < end)
*pos++ = curr++;
next += step;
progress(column->size(), column->byteSize());
return { Columns {std::move(column)}, block_size };
}
private:
UInt64 block_size;
UInt64 next;
UInt64 step;
static Block createHeader()
{
return { ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "number") };
}
};
struct NumbersMultiThreadedState
{
std::atomic<UInt64> counter;
explicit NumbersMultiThreadedState(UInt64 offset) : counter(offset) {}
};
using NumbersMultiThreadedStatePtr = std::shared_ptr<NumbersMultiThreadedState>;
class NumbersMultiThreadedSource : public ISource
{
public:
NumbersMultiThreadedSource(NumbersMultiThreadedStatePtr state_, UInt64 block_size_, UInt64 max_counter_)
: ISource(createHeader())
, state(std::move(state_))
, block_size(block_size_)
, max_counter(max_counter_) {}
String getName() const override { return "NumbersMt"; }
protected:
Chunk generate() override
{
if (block_size == 0)
return {};
UInt64 curr = state->counter.fetch_add(block_size, std::memory_order_relaxed);
if (curr >= max_counter)
return {};
if (curr + block_size > max_counter)
block_size = max_counter - curr;
auto column = ColumnUInt64::create(block_size);
ColumnUInt64::Container & vec = column->getData();
UInt64 * pos = vec.data();
UInt64 * end = &vec[block_size];
while (pos < end)
*pos++ = curr++;
progress(column->size(), column->byteSize());
return { Columns {std::move(column)}, block_size };
}
private:
NumbersMultiThreadedStatePtr state;
UInt64 block_size;
UInt64 max_counter;
static Block createHeader()
{
return { ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "number") };
}
};
}
StorageSystemNumbers::StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, std::optional<UInt64> limit_, UInt64 offset_, bool even_distribution_)
: IStorage(table_id), multithreaded(multithreaded_), even_distribution(even_distribution_), limit(limit_), offset(offset_)
{ {
StorageInMemoryMetadata storage_metadata; StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription({{"number", std::make_shared<DataTypeUInt64>()}})); storage_metadata.setColumns(ColumnsDescription({{"number", std::make_shared<DataTypeUInt64>()}}));
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
} }
Pipe StorageSystemNumbers::read( void StorageSystemNumbers::read(
QueryPlan & query_plan,
const Names & column_names, const Names & column_names,
const StorageSnapshotPtr & storage_snapshot, const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info, SelectQueryInfo & query_info,
ContextPtr /*context*/, ContextPtr context,
QueryProcessingStage::Enum /*processed_stage*/, QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size, size_t max_block_size,
size_t num_streams) size_t num_streams)
{ {
storage_snapshot->check(column_names); query_plan.addStep(std::make_unique<ReadFromSystemNumbersStep>(
column_names, shared_from_this(), storage_snapshot, query_info, std::move(context), max_block_size, num_streams));
if (limit && *limit < max_block_size)
{
max_block_size = static_cast<size_t>(*limit);
multithreaded = false;
}
if (!multithreaded)
num_streams = 1;
Pipe pipe;
if (num_streams > 1 && !even_distribution && limit)
{
auto state = std::make_shared<NumbersMultiThreadedState>(offset);
UInt64 max_counter = offset + *limit;
for (size_t i = 0; i < num_streams; ++i)
{
auto source = std::make_shared<NumbersMultiThreadedSource>(state, max_block_size, max_counter);
if (i == 0)
{
auto rows_appr = *limit;
if (query_info.limit > 0 && query_info.limit < rows_appr)
rows_appr = query_info.limit;
source->addTotalRowsApprox(rows_appr);
}
pipe.addSource(std::move(source));
}
return pipe;
}
for (size_t i = 0; i < num_streams; ++i)
{
auto source = std::make_shared<NumbersSource>(max_block_size, offset + i * max_block_size, num_streams * max_block_size);
if (limit && i == 0)
{
auto rows_appr = *limit;
if (query_info.limit > 0 && query_info.limit < rows_appr)
rows_appr = query_info.limit;
source->addTotalRowsApprox(rows_appr);
}
pipe.addSource(std::move(source));
}
if (limit)
{
size_t i = 0;
/// This formula is how to split 'limit' elements to 'num_streams' chunks almost uniformly.
pipe.addSimpleTransform([&](const Block & header)
{
++i;
return std::make_shared<LimitTransform>(
header, *limit * i / num_streams - *limit * (i - 1) / num_streams, 0);
});
}
return pipe;
} }
} }

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <optional> #include <optional>
#include <Parsers/ASTIdentifier.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
@ -15,38 +16,62 @@ class Context;
* From this table, you can read all natural numbers, starting from 0 (to 2^64 - 1, and then again). * From this table, you can read all natural numbers, starting from 0 (to 2^64 - 1, and then again).
* *
* You could also specify a limit (how many numbers to give). * You could also specify a limit (how many numbers to give).
*
* How to generate numbers?
*
* 1. First try a smart fashion:
*
* In this fashion we try to push filters and limit down to scanning.
* Firstly extract plain ranges(no overlapping and ordered) by filter expressions.
*
* For example:
* where (numbers > 1 and numbers < 3) or (numbers in (4, 6)) or (numbers > 7 and numbers < 9)
*
* We will get ranges
* (1, 3), [4, 4], [6, 6], (7, 9)
*
* Then split the ranges evenly to one or multi-streams. With this way we will get result without large scanning.
*
* 2. If fail to extract plain ranges, fall back to ordinary scanning.
*
* If multithreaded is specified, numbers will be generated in several streams * If multithreaded is specified, numbers will be generated in several streams
* (and result could be out of order). If both multithreaded and limit are specified, * (and result could be out of order). If both multithreaded and limit are specified,
* the table could give you not exactly 1..limit range, but some arbitrary 'limit' numbers. * the table could give you not exactly 1..limit range, but some arbitrary 'limit' numbers.
*
* In multithreaded case, if even_distributed is False, implementation with atomic is used,
* and result is always in [0 ... limit - 1] range.
*/ */
class StorageSystemNumbers final : public IStorage class StorageSystemNumbers final : public IStorage
{ {
public: public:
/// If even_distribution is true, numbers are distributed evenly between streams.
/// Otherwise, streams concurrently increment atomic. /// Otherwise, streams concurrently increment atomic.
StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, std::optional<UInt64> limit_ = std::nullopt, UInt64 offset_ = 0, bool even_distribution_ = true); StorageSystemNumbers(const StorageID & table_id, bool multithreaded_, std::optional<UInt64> limit_ = std::nullopt, UInt64 offset_ = 0);
std::string getName() const override { return "SystemNumbers"; } std::string getName() const override { return "SystemNumbers"; }
Pipe read( void read(
QueryPlan & query_plan,
const Names & column_names, const Names & column_names,
const StorageSnapshotPtr & storage_snapshot, const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info, SelectQueryInfo & query_info,
ContextPtr context, ContextPtr context,
QueryProcessingStage::Enum processed_stage, QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size, size_t max_block_size,
size_t num_streams) override; size_t num_streams) override;
bool hasEvenlyDistributedRead() const override { return true; } bool hasEvenlyDistributedRead() const override { return true; }
bool isSystemStorage() const override { return true; } bool isSystemStorage() const override { return true; }
bool supportsTransactions() const override { return true; } bool supportsTransactions() const override { return true; }
bool supportsIndexForIn() const override { return true; }
bool mayBenefitFromIndexForIn(
const ASTPtr & left_in_operand, ContextPtr /* query_context */, const StorageMetadataPtr & /* metadata_snapshot */) const override
{
return left_in_operand->as<ASTIdentifier>() && left_in_operand->getColumnName() == "number";
}
private: private:
friend class ReadFromSystemNumbersStep;
bool multithreaded; bool multithreaded;
bool even_distribution;
std::optional<UInt64> limit; std::optional<UInt64> limit;
UInt64 offset; UInt64 offset;
}; };

View File

@ -63,7 +63,7 @@ StoragePtr TableFunctionNumbers<multithreaded>::executeImpl(const ASTPtr & ast_f
UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0; UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0;
UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]);
auto res = std::make_shared<StorageSystemNumbers>(StorageID(getDatabaseName(), table_name), multithreaded, length, offset, false); auto res = std::make_shared<StorageSystemNumbers>(StorageID(getDatabaseName(), table_name), multithreaded, length, offset);
res->startup(); res->startup();
return res; return res;
} }

View File

@ -388,8 +388,6 @@ def test_progress():
rows=8, rows=8,
blocks=4, blocks=4,
allocated_bytes=1092, allocated_bytes=1092,
applied_limit=True,
rows_before_limit=8,
) )
), ),
] ]

View File

@ -0,0 +1,244 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance("node", main_configs=[])
@pytest.fixture(scope="module")
def started_cluster(request):
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def check_read_rows(query_id, read_rows):
node.query("SYSTEM FLUSH LOGS")
real_read_rows = node.query(
f"""SELECT read_rows
FROM system.query_log
WHERE
type = 'QueryFinish' and
query_id = '{query_id}'
ORDER BY initial_query_start_time_microseconds DESC
LIMIT 1"""
)
assert real_read_rows == str(read_rows) + "\n"
def test_simple_range(started_cluster):
response = node.query(
"SELECT * FROM system.numbers WHERE number=1 FORMAT Values",
query_id="test_equal",
)
assert response == "(1)"
check_read_rows("test_equal", 1)
response = node.query(
"SELECT * FROM system.numbers WHERE number > 1 and number < 6 FORMAT Values",
query_id="test_single_range",
)
assert response == "(2),(3),(4),(5)"
check_read_rows("test_single_range", 4)
def test_between(started_cluster):
response = node.query(
"SELECT * FROM system.numbers WHERE number between 1 and 6 FORMAT Values",
query_id="test_between",
)
assert response == "(1),(2),(3),(4),(5),(6)"
check_read_rows("test_between", 6)
def test_blank_range(started_cluster):
response = node.query(
"SELECT * FROM system.numbers WHERE number < 1 and number > 6 FORMAT Values",
query_id="test_blank_range",
)
assert response == ""
check_read_rows("test_blank_range", 0)
def test_in(started_cluster):
response = node.query(
"SELECT * FROM system.numbers WHERE number in (2, 3) FORMAT Values",
query_id="test_in_simple",
)
assert response == "(2),(3)"
check_read_rows("test_in_simple", 2)
response = node.query(
"SELECT * FROM system.numbers WHERE number in (2, 3, 3) FORMAT Values",
query_id="test_in_with_duplicated_values",
)
assert response == "(2),(3)"
check_read_rows("test_in_with_duplicated_values", 2)
response = node.query(
"SELECT * FROM system.numbers WHERE number in (2, 3, 1) FORMAT Values",
query_id="test_in_with_unordered_values",
)
assert response == "(1),(2),(3)"
check_read_rows("test_in_with_unordered_values", 3)
response = node.query(
"SELECT * FROM system.numbers WHERE number in (1, 2, 5) FORMAT Values",
query_id="test_in_multiple_ranges",
)
assert response == "(1),(2),(5)"
check_read_rows("test_in_multiple_ranges", 3)
def test_not_in(started_cluster):
response = node.query(
"SELECT * FROM system.numbers WHERE number not in (2, 3) limit 3 FORMAT Values",
query_id="test_not_in",
)
assert response == "(0),(1),(4)"
check_read_rows("test_not_in", 3)
response = node.query(
"SELECT * FROM system.numbers WHERE number not in (2, 4, 5) limit 4 FORMAT Values",
query_id="test_not_in_multiple_ranges",
)
assert response == "(0),(1),(3),(6)"
check_read_rows("test_not_in_multiple_ranges", 4)
def test_and(started_cluster):
response = node.query(
"SELECT * FROM system.numbers WHERE number in (2, 4) and number > 2 FORMAT Values",
query_id="test_and",
)
assert response == "(4)"
check_read_rows("test_and", 1)
def test_or(started_cluster):
response = node.query(
"""SELECT
*
FROM
system.numbers
WHERE
(number > 1 and number < 3) or (number in (4, 6)) or (number > 7 and number < 9)
FORMAT Values""",
query_id="test_simple_or",
)
assert response == "(2),(4),(6),(8)"
check_read_rows("test_simple_or", 4)
response = node.query(
"SELECT * FROM system.numbers WHERE (number > 1 and number < 3) or (number < 6) FORMAT Values",
query_id="test_or_with_overlapping_ranges",
)
assert response == "(0),(1),(2),(3),(4),(5)"
check_read_rows("test_or_with_overlapping_ranges", 6)
def test_not(started_cluster):
response = node.query(
"SELECT * FROM system.numbers WHERE not (number > 1 and number < 3) limit 5 FORMAT Values",
query_id="test_not",
)
assert response == "(0),(1),(3),(4),(5)"
check_read_rows("test_not", 5)
def test_true_or_false(started_cluster):
response = node.query(
"SELECT * FROM system.numbers WHERE number < 3 and 1 limit 5 FORMAT Values",
query_id="test_true",
)
assert response == "(0),(1),(2)"
check_read_rows("test_true", 3)
response = node.query(
"SELECT * FROM system.numbers WHERE number < 3 and 0 FORMAT Values",
query_id="test_false",
)
assert response == ""
check_read_rows("test_false", 0)
def test_limit(started_cluster):
response = node.query(
"SELECT * FROM system.numbers WHERE number > 2 limit 1 FORMAT Values",
query_id="test_simple_limit",
)
assert response == "(3)"
check_read_rows("test_simple_limit", 1)
response = node.query(
"SELECT * FROM system.numbers WHERE number not in (2, 3) limit 1 FORMAT Values",
query_id="test_limit_with_multi_ranges",
)
assert response == "(0)"
check_read_rows("test_limit_with_multi_ranges", 1)
response = node.query(
"SELECT * FROM system.numbers WHERE number not in (2, 3) limit 1, 2 FORMAT Values",
query_id="test_limit_with_offset",
)
assert response == "(1),(4)"
check_read_rows("test_limit_with_offset", 3)
def test_subquery(started_cluster):
response = node.query(
"""SELECT
*
FROM
(select * FROM system.numbers WHERE number < 2) AS n
FORMAT Values""",
query_id="test_subquery",
)
assert response == "(0),(1)"
check_read_rows("test_subquery", 2)
def test_multi_streams(started_cluster):
response = node.query(
"""SELECT
*
FROM
system.numbers_mt
WHERE
number > 1 and number < 7
ORDER BY
number
FORMAT Values
settings max_block_size=2""",
query_id="test_multi_streams",
)
assert response == "(2),(3),(4),(5),(6)"
check_read_rows("test_multi_streams", 5)
response = node.query(
"""SELECT
*
FROM
system.numbers_mt
WHERE
(number > 1 and number < 3) or (number in (4, 6)) or (number > 7 and number < 10)
ORDER BY
number
FORMAT Values
settings max_block_size=2""",
query_id="test_multi_streams_with_multi_ranges",
)
assert response == "(2),(4),(6),(8),(9)"
check_read_rows("test_multi_streams", 5)
def test_overflow(started_cluster):
response = node.query(
"SELECT number FROM numbers(18446744073709551614, 5) FORMAT Values",
query_id="test_overflow",
)
assert response == "(18446744073709551614),(18446744073709551615),(0),(1),(2)"
check_read_rows("test_overflow", 5)

View File

@ -1,4 +1,5 @@
SET max_block_size = 1000; SET max_block_size = 1000;
SET max_threads = 10;
SELECT SELECT
groupUniqArray(blockSize()), groupUniqArray(blockSize()),
uniqExact(rowNumberInAllBlocks()), uniqExact(rowNumberInAllBlocks()),

View File

@ -24,4 +24,4 @@
7 7
8 8
9 9
< X-ClickHouse-Summary: {"read_rows":"10","read_bytes":"80","written_rows":"10","written_bytes":"40","total_rows_to_read":"0","result_rows":"10","result_bytes":"40"} < X-ClickHouse-Summary: {"read_rows":"10","read_bytes":"80","written_rows":"10","written_bytes":"40","total_rows_to_read":"10","result_rows":"10","result_bytes":"40"}

View File

@ -63,8 +63,6 @@ Totals:
"s": "6" "s": "6"
}, },
"rows": 4, "rows": 4
"rows_before_limit_at_least": 4
} }
-- --

View File

@ -70,7 +70,5 @@ UInt64
["9"] ["9"]
], ],
"rows": 10, "rows": 10
"rows_before_limit_at_least": 10
} }

View File

@ -921,7 +921,7 @@ Expression ((Project names + Projection))
Window (Window step for window \'PARTITION BY p_0 ORDER BY o_1 ASC\') Window (Window step for window \'PARTITION BY p_0 ORDER BY o_1 ASC\')
Sorting (Sorting for window \'PARTITION BY p_0 ORDER BY o_1 ASC\') Sorting (Sorting for window \'PARTITION BY p_0 ORDER BY o_1 ASC\')
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
explain select explain select
count(*) over (order by o, number), count(*) over (order by o, number),
count(*) over (order by number) count(*) over (order by number)
@ -936,7 +936,7 @@ Expression ((Project names + Projection))
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) [lifted up part]) Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))) [lifted up part])
Sorting (Sorting for window \'ORDER BY number_1 ASC\') Sorting (Sorting for window \'ORDER BY number_1 ASC\')
Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) Expression ((Before WINDOW + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- A test case for the sort comparator found by fuzzer. -- A test case for the sort comparator found by fuzzer.
SELECT SELECT
max(number) OVER (ORDER BY number DESC NULLS FIRST), max(number) OVER (ORDER BY number DESC NULLS FIRST),

View File

@ -1,3 +1,5 @@
"detail": "clickhouse",
"row_number": "999997"
}, },
{ {
"datetime": "2020-12-12", "datetime": "2020-12-12",
@ -19,5 +21,3 @@
"rows": 1000000, "rows": 1000000,
"rows_before_limit_at_least": 1048080,

View File

@ -1,3 +1,5 @@
"detail": "clickhouse",
"row_number": "99997"
}, },
{ {
"datetime": "2020-12-12", "datetime": "2020-12-12",
@ -19,5 +21,3 @@
"rows": 100000, "rows": 100000,
"rows_before_limit_at_least": 131010,

View File

@ -1,3 +1,5 @@
"detail": "clickhouse",
"row_number": "999997"
}, },
{ {
"datetime": "2020-12-12", "datetime": "2020-12-12",
@ -19,5 +21,3 @@
"rows": 1000000, "rows": 1000000,
"rows_before_limit_at_least": 1048080,

View File

@ -104,8 +104,8 @@ ExpressionTransform
Copy × 3 1 → 2 Copy × 3 1 → 2
(Expression) (Expression)
ExpressionTransform × 3 ExpressionTransform × 3
(ReadFromStorage) (ReadFromSystemNumbers)
NumbersMt × 3 0 → 1 NumbersRange × 3 0 → 1
4999500000 10000 4999500000 10000
4999510000 10000 4999510000 10000
4999520000 10000 4999520000 10000

View File

@ -6,7 +6,7 @@ Expression
Union Union
Sorting Sorting
Expression Expression
ReadFromStorage ReadFromSystemNumbers
ReadFromRemote ReadFromRemote
explain description=0 select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1; explain description=0 select * from remote('127.{1,2}', view(select * from numbers(1e6))) order by number limit 10 settings distributed_push_down_limit=1;
Expression Expression
@ -15,5 +15,5 @@ Expression
Union Union
Sorting Sorting
Expression Expression
ReadFromStorage ReadFromSystemNumbers
ReadFromRemote ReadFromRemote

View File

@ -5,7 +5,7 @@ Expression (Projection)
Union Union
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
ReadFromRemote (Read from remote replica) ReadFromRemote (Read from remote replica)
explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized
Union Union
@ -13,14 +13,14 @@ Union
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
ReadFromRemote (Read from remote replica) ReadFromRemote (Read from remote replica)
explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized
Expression (Projection) Expression (Projection)
LimitBy LimitBy
Union Union
Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))) Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression Expression
ReadFromRemote (Read from remote replica) ReadFromRemote (Read from remote replica)
explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized
@ -28,7 +28,7 @@ Union
Expression (Projection) Expression (Projection)
LimitBy LimitBy
Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))) Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
ReadFromRemote (Read from remote replica) ReadFromRemote (Read from remote replica)
explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized explain select distinct k1 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized
Expression (Projection) Expression (Projection)
@ -38,7 +38,7 @@ Expression (Projection)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
ReadFromRemote (Read from remote replica) ReadFromRemote (Read from remote replica)
explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized explain select distinct k1, k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized
Expression (Projection) Expression (Projection)
@ -48,7 +48,7 @@ Expression (Projection)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
ReadFromRemote (Read from remote replica) ReadFromRemote (Read from remote replica)
explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- not optimized
Expression (Projection) Expression (Projection)
@ -59,7 +59,7 @@ Expression (Projection)
Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) [lifted up part]) Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) [lifted up part])
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
ReadFromRemote (Read from remote replica) ReadFromRemote (Read from remote replica)
explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized
Expression (Projection) Expression (Projection)
@ -69,5 +69,5 @@ Expression (Projection)
Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) [lifted up part])) Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) [lifted up part]))
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))) Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
ReadFromRemote (Read from remote replica) ReadFromRemote (Read from remote replica)

View File

@ -1,7 +1,7 @@
do not print any ProfileEvents packets do not print any ProfileEvents packets
0 0
print only last (and also number of rows to provide more info in case of failures) print only last (and also number of rows to provide more info in case of failures)
[ 0 ] SelectedRows: 131010 (increment) [ 0 ] SelectedRows: 100000 (increment)
regression test for incorrect filtering out snapshots regression test for incorrect filtering out snapshots
0 0
regression test for overlap profile events snapshots between queries regression test for overlap profile events snapshots between queries

View File

@ -1,2 +1 @@
finished default_TEST02132KILL_QUERY1 default select (SELECT max(number) from system.numbers) + 1; finished default_TEST02132KILL_QUERY1 default select (SELECT max(number) from system.numbers) + 1;
finished default_TEST02132KILL_QUERY2 default SELECT (SELECT number FROM system.numbers WHERE number = 1000000000000);

View File

@ -14,9 +14,4 @@ QUERY_1_ID="${CLICKHOUSE_DATABASE}_TEST02132KILL_QUERY1"
wait_for_query_to_start "${QUERY_1_ID}" wait_for_query_to_start "${QUERY_1_ID}"
${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_1_ID}' SYNC" ${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_1_ID}' SYNC"
QUERY_2_ID="${CLICKHOUSE_DATABASE}_TEST02132KILL_QUERY2"
(${CLICKHOUSE_CLIENT} --query_id="${QUERY_2_ID}" --query='SELECT (SELECT number FROM system.numbers WHERE number = 1000000000000);' 2>&1 | grep -q "Code: 394." || echo 'FAIL') &
wait_for_query_to_start "${QUERY_2_ID}"
${CLICKHOUSE_CLIENT} --query="KILL QUERY WHERE query_id='${QUERY_2_ID}' SYNC"
wait wait

View File

@ -1,6 +1,6 @@
< X-ClickHouse-Progress: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} < X-ClickHouse-Progress: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"}
< X-ClickHouse-Progress: {"read_rows":"65505","read_bytes":"524040","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} < X-ClickHouse-Progress: {"read_rows":"65505","read_bytes":"524040","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"}
< X-ClickHouse-Progress: {"read_rows":"131010","read_bytes":"1048080","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} < X-ClickHouse-Progress: {"read_rows":"100000","read_bytes":"800000","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"}
< X-ClickHouse-Progress: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"} < X-ClickHouse-Progress: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"0","result_bytes":"0"}
< X-ClickHouse-Progress: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"272"} < X-ClickHouse-Progress: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"272"}
< X-ClickHouse-Summary: {"read_rows":"131011","read_bytes":"1048081","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"272"} < X-ClickHouse-Summary: {"read_rows":"100001","read_bytes":"800001","written_rows":"0","written_bytes":"0","total_rows_to_read":"100000","result_rows":"1","result_bytes":"272"}

View File

@ -17,8 +17,6 @@
"rows": 1, "rows": 1,
"rows_before_limit_at_least": 100,
"statistics": "statistics":
{ {
"rows_read": 100, "rows_read": 100,
@ -44,7 +42,7 @@
"statistics": "statistics":
{ {
"rows_read": 131011, "rows_read": 100001,
"bytes_read": 1048081 "bytes_read": 800001
} }
} }

View File

@ -6,7 +6,7 @@ ExpressionTransform 1000000 8000000 1000000 8000000
LazyOutputFormat 1 8 0 0 LazyOutputFormat 1 8 0 0
LimitsCheckingTransform 1 8 1 8 LimitsCheckingTransform 1 8 1 8
NullSource 0 0 0 0 NullSource 0 0 0 0
NumbersMt 0 0 1000000 8000000 NumbersRange 0 0 1000000 8000000
Resize 1 8 1 8 Resize 1 8 1 8
Resize 1 8 1 8 Resize 1 8 1 8
1 1

View File

@ -6,12 +6,12 @@ ExpressionTransform
ExpressionTransform ExpressionTransform
(Limit) (Limit)
Limit Limit
(ReadFromStorage) (ReadFromSystemNumbers)
Numbers 0 → 1 NumbersRange 0 → 1
(Expression) (Expression)
FillingRightJoinSide FillingRightJoinSide
ExpressionTransform ExpressionTransform
(Limit) (Limit)
Limit Limit
(ReadFromStorage) (ReadFromSystemNumbers)
Numbers 0 → 1 NumbersRange 0 → 1

View File

@ -14,9 +14,8 @@ ExpressionTransform × 16
AggregatingTransform AggregatingTransform
(Expression) (Expression)
ExpressionTransform ExpressionTransform
(ReadFromStorage) (ReadFromSystemNumbers)
Limit NumbersRange 0 → 1
Numbers 0 → 1
explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number; explain pipeline select * from (select * from numbers_mt(1e8) group by number) group by number;
(Expression) (Expression)
ExpressionTransform × 16 ExpressionTransform × 16
@ -31,8 +30,8 @@ ExpressionTransform × 16
AggregatingTransform × 16 AggregatingTransform × 16
(Expression) (Expression)
ExpressionTransform × 16 ExpressionTransform × 16
(ReadFromStorage) (ReadFromSystemNumbers)
NumbersMt × 16 0 → 1 NumbersRange × 16 0 → 1
explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number; explain pipeline select * from (select * from numbers_mt(1e8) group by number) order by number;
(Expression) (Expression)
ExpressionTransform ExpressionTransform
@ -48,8 +47,8 @@ ExpressionTransform
AggregatingTransform × 16 AggregatingTransform × 16
(Expression) (Expression)
ExpressionTransform × 16 ExpressionTransform × 16
(ReadFromStorage) (ReadFromSystemNumbers)
NumbersMt × 16 0 → 1 NumbersRange × 16 0 → 1
explain pipeline select number from remote('127.0.0.{1,2,3}', system, numbers_mt) group by number settings distributed_aggregation_memory_efficient = 1; explain pipeline select number from remote('127.0.0.{1,2,3}', system, numbers_mt) group by number settings distributed_aggregation_memory_efficient = 1;
(Expression) (Expression)
ExpressionTransform × 16 ExpressionTransform × 16
@ -65,8 +64,8 @@ ExpressionTransform × 16
AggregatingTransform × 16 AggregatingTransform × 16
(Expression) (Expression)
ExpressionTransform × 16 ExpressionTransform × 16
(ReadFromStorage) (ReadFromSystemNumbers)
Numbers × 16 0 → 1 NumbersRange × 16 0 → 1
(ReadFromRemote) (ReadFromRemote)
explain pipeline select number from remote('127.0.0.{1,2,3}', system, numbers_mt) group by number settings distributed_aggregation_memory_efficient = 0; explain pipeline select number from remote('127.0.0.{1,2,3}', system, numbers_mt) group by number settings distributed_aggregation_memory_efficient = 0;
(Expression) (Expression)
@ -81,8 +80,8 @@ ExpressionTransform × 16
AggregatingTransform × 16 AggregatingTransform × 16
(Expression) (Expression)
ExpressionTransform × 16 ExpressionTransform × 16
(ReadFromStorage) (ReadFromSystemNumbers)
Numbers × 16 0 → 1 NumbersRange × 16 0 → 1
(ReadFromRemote) (ReadFromRemote)
-- { echoOn } -- { echoOn }

View File

@ -7,7 +7,7 @@ drop table if exists t_mv;
create table t (a UInt64) Engine = Null; create table t (a UInt64) Engine = Null;
create materialized view t_mv Engine = Null AS select now() as ts, max(a) from t group by ts; create materialized view t_mv Engine = Null AS select now() as ts, max(a) from t group by ts;
insert into t select * from numbers_mt(10e6) settings max_threads = 16, max_insert_threads=16; insert into t select * from numbers_mt(10e6) settings max_threads = 16, max_insert_threads=16, max_block_size=100000;
system flush logs; system flush logs;
select arrayUniq(thread_ids)>=16 from system.query_log where select arrayUniq(thread_ids)>=16 from system.query_log where

View File

@ -2,20 +2,20 @@
1 1
Expression ((Project names + (Projection + Change column names to column identifiers))) Expression ((Project names + (Projection + Change column names to column identifiers)))
Limit (preliminary LIMIT (without OFFSET)) Limit (preliminary LIMIT (without OFFSET))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Project names + (Projection + Change column names to column identifiers))) Expression ((Project names + (Projection + Change column names to column identifiers)))
Limit (preliminary LIMIT (without OFFSET)) Limit (preliminary LIMIT (without OFFSET))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
(Expression) (Expression)
ExpressionTransform ExpressionTransform
(Limit) (Limit)
Limit Limit
(ReadFromStorage) (ReadFromSystemNumbers)
Numbers 0 → 1 NumbersRange 0 → 1
(Expression) (Expression)
ExpressionTransform ExpressionTransform
(Limit) (Limit)
Limit Limit
(ReadFromStorage) (ReadFromSystemNumbers)
Numbers 0 → 1 NumbersRange 0 → 1
1 1

View File

@ -7,7 +7,7 @@ Expression (Projection)
Expression ((Before ORDER BY + Projection)) Expression ((Before ORDER BY + Projection))
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- Enabled query_plan_remove_redundant_sorting -- Enabled query_plan_remove_redundant_sorting
-- ORDER BY removes ORDER BY clauses in subqueries -- ORDER BY removes ORDER BY clauses in subqueries
-- query -- query
@ -28,7 +28,7 @@ ORDER BY number ASC
Expression (Projection) Expression (Projection)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + (Before ORDER BY + (Projection + Before ORDER BY))))) Expression ((Before ORDER BY + (Projection + (Before ORDER BY + (Projection + Before ORDER BY)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -55,7 +55,7 @@ Expression (Projection)
Filling Filling
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + Before ORDER BY))) Expression ((Before ORDER BY + (Projection + Before ORDER BY)))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -84,7 +84,7 @@ Expression (Projection)
Expression (Before LIMIT BY) Expression (Before LIMIT BY)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + Before ORDER BY))) Expression ((Before ORDER BY + (Projection + Before ORDER BY)))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -119,11 +119,11 @@ Expression ((Projection + Before ORDER BY))
Expression ((Before JOIN + Projection)) Expression ((Before JOIN + Projection))
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + Before ORDER BY))) Expression ((Before ORDER BY + (Projection + Before ORDER BY)))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + Projection))) Expression ((Joined actions + (Rename joined columns + Projection)))
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + Before ORDER BY))) Expression ((Before ORDER BY + (Projection + Before ORDER BY)))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 2 0 2
0 1 0 1
@ -165,9 +165,9 @@ Expression (Projection)
Expression (Before ORDER BY) Expression (Before ORDER BY)
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression ((Before JOIN + (Projection + (Before ORDER BY + (Projection + Before ORDER BY))))) Expression ((Before JOIN + (Projection + (Before ORDER BY + (Projection + Before ORDER BY)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + (Projection + (Before ORDER BY + (Projection + Before ORDER BY)))))) Expression ((Joined actions + (Rename joined columns + (Projection + (Before ORDER BY + (Projection + Before ORDER BY))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0 0 0
0 1 0 1
@ -197,7 +197,7 @@ GROUP BY number
Expression ((Projection + Before ORDER BY)) Expression ((Projection + Before ORDER BY))
Aggregating Aggregating
Expression ((Before GROUP BY + (Projection + (Before ORDER BY + (Projection + Before ORDER BY))))) Expression ((Before GROUP BY + (Projection + (Before ORDER BY + (Projection + Before ORDER BY)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -221,7 +221,7 @@ GROUP BY number
Expression ((Projection + Before ORDER BY)) Expression ((Projection + Before ORDER BY))
Aggregating Aggregating
Expression ((Before GROUP BY + (Projection + (Before ORDER BY + (Projection + Before ORDER BY))))) Expression ((Before GROUP BY + (Projection + (Before ORDER BY + (Projection + Before ORDER BY)))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -239,7 +239,7 @@ FROM
Expression ((Projection + Before ORDER BY)) Expression ((Projection + Before ORDER BY))
Aggregating Aggregating
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
45 45
-- check that optimization is applied recursively to subqueries as well -- check that optimization is applied recursively to subqueries as well
@ -264,7 +264,7 @@ Expression (Projection)
Expression ((Before ORDER BY + (Projection + Before ORDER BY))) Expression ((Before ORDER BY + (Projection + Before ORDER BY)))
Aggregating Aggregating
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -290,7 +290,7 @@ Expression (Projection)
Expression ((Before ORDER BY + (Projection + Before ORDER BY))) Expression ((Before ORDER BY + (Projection + Before ORDER BY)))
Aggregating Aggregating
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -320,7 +320,7 @@ Expression ((Projection + (Before ORDER BY + )))
Aggregating Aggregating
Filter Filter
Filter (( + (Before GROUP BY + (Projection + (Before ORDER BY + (Projection + Before ORDER BY)))))) Filter (( + (Before GROUP BY + (Projection + (Before ORDER BY + (Projection + Before ORDER BY))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
2 2
1 1
@ -346,7 +346,7 @@ Expression (Projection)
Expression ((Before ORDER BY + (Projection + (Before ORDER BY + (Projection + Before ORDER BY))))) Expression ((Before ORDER BY + (Projection + (Before ORDER BY + (Projection + Before ORDER BY)))))
Aggregating Aggregating
Expression (Before GROUP BY) Expression (Before GROUP BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -467,7 +467,7 @@ Expression (Projection)
Expression (Projection) Expression (Projection)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0 0 0
1 0 1 0
@ -496,7 +496,7 @@ Expression (Projection)
Expression (Projection) Expression (Projection)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
9 11 9 11
8 10 8 10

View File

@ -7,7 +7,7 @@ Expression (Project names)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + Project names)))) Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + Project names))))
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + Change column names to column identifiers))) Expression ((Before ORDER BY + (Projection + Change column names to column identifiers)))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- Enabled query_plan_remove_redundant_sorting -- Enabled query_plan_remove_redundant_sorting
-- ORDER BY removes ORDER BY clauses in subqueries -- ORDER BY removes ORDER BY clauses in subqueries
-- query -- query
@ -28,7 +28,7 @@ ORDER BY number ASC
Expression (Project names) Expression (Project names)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))))) Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -55,7 +55,7 @@ Expression (Project names)
Filling Filling
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))) Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -84,7 +84,7 @@ Expression (Project names)
Expression (Before LIMIT BY) Expression (Before LIMIT BY)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))) Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -119,11 +119,11 @@ Expression ((Project names + (Projection + DROP unused columns after JOIN)))
Expression ((Change column names to column identifiers + Project names)) Expression ((Change column names to column identifiers + Project names))
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))) Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Change column names to column identifiers + Project names)) Expression ((Change column names to column identifiers + Project names))
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))) Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 2 0 2
0 1 0 1
@ -165,9 +165,9 @@ Expression (Project names)
Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN))) Expression ((Before ORDER BY + (Projection + DROP unused columns after JOIN)))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))) Expression ((Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0 0 0
0 1 0 1
@ -197,7 +197,7 @@ GROUP BY number
Expression ((Project names + Projection)) Expression ((Project names + Projection))
Aggregating Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -223,7 +223,7 @@ Expression ((Project names + Projection))
Expression ((Before GROUP BY + (Change column names to column identifiers + Project names))) Expression ((Before GROUP BY + (Change column names to column identifiers + Project names)))
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))) Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -241,7 +241,7 @@ FROM
Expression ((Project names + Projection)) Expression ((Project names + Projection))
Aggregating Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
45 45
-- check that optimization is applied recursively to subqueries as well -- check that optimization is applied recursively to subqueries as well
@ -266,7 +266,7 @@ Expression (Project names)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + Projection))))) Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + Projection)))))
Aggregating Aggregating
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -294,7 +294,7 @@ Expression (Project names)
Expression ((Before GROUP BY + (Change column names to column identifiers + Project names))) Expression ((Before GROUP BY + (Change column names to column identifiers + Project names)))
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + Change column names to column identifiers))) Expression ((Before ORDER BY + (Projection + Change column names to column identifiers)))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -326,7 +326,7 @@ Expression ((Project names + Projection))
Expression ((Before GROUP BY + (Change column names to column identifiers + Project names))) Expression ((Before GROUP BY + (Change column names to column identifiers + Project names)))
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers))))))) Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + Change column names to column identifiers)))))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
2 2
1 1
@ -352,7 +352,7 @@ Expression (Project names)
Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + Projection)))))))))) Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + (Projection + (Change column names to column identifiers + (Project names + (Before ORDER BY + Projection))))))))))
Aggregating Aggregating
Expression ((Before GROUP BY + Change column names to column identifiers)) Expression ((Before GROUP BY + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -477,7 +477,7 @@ Expression (Project names)
Expression (Before ORDER BY) Expression (Before ORDER BY)
Expression (Projection) Expression (Projection)
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0 0 0
1 0 1 0
@ -509,7 +509,7 @@ Expression (Project names)
Expression (Before ORDER BY) Expression (Before ORDER BY)
Expression (Projection) Expression (Projection)
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
9 11 9 11
8 10 8 10

View File

@ -9,7 +9,7 @@ Expression (Projection)
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- Enabled query_plan_remove_redundant_distinct -- Enabled query_plan_remove_redundant_distinct
-- DISTINCT is only in most inner subquery -- DISTINCT is only in most inner subquery
-- query -- query
@ -28,7 +28,7 @@ Expression ((Projection + (Before ORDER BY + (Projection + (Before ORDER BY + Pr
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -52,12 +52,12 @@ Expression (Projection)
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (( + Projection)) Expression (( + Projection))
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -83,12 +83,12 @@ Expression (Projection)
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + Projection))) Expression ((Joined actions + (Rename joined columns + Projection)))
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0 0 0
0 1 0 1
@ -111,7 +111,7 @@ Expression ((Projection + (Before ORDER BY + (Projection + (Before ORDER BY + Pr
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0 0 0
1 2 1 2
@ -133,7 +133,7 @@ Expression ((Projection + (Before ORDER BY + (Projection + (Before ORDER BY + Pr
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
2 0 0 2 0 0
2 1 2 2 1 2
@ -229,9 +229,9 @@ Expression ((Projection + (Before ORDER BY + (Projection + Before ORDER BY))))
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Before JOIN) Expression (Before JOIN)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -262,9 +262,9 @@ Expression (Projection)
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Before JOIN) Expression (Before JOIN)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
12 12
-- GROUP BY WITH ROLLUP before DISTINCT with on different columns => do _not_ remove DISTINCT -- GROUP BY WITH ROLLUP before DISTINCT with on different columns => do _not_ remove DISTINCT
@ -294,9 +294,9 @@ Expression (Projection)
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Before JOIN) Expression (Before JOIN)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
12 12
36 36
@ -324,9 +324,9 @@ Expression ((Projection + (Before ORDER BY + (Projection + Before ORDER BY))))
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Before JOIN) Expression (Before JOIN)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -359,9 +359,9 @@ Expression (Projection)
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Before JOIN) Expression (Before JOIN)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
12 12
36 36
@ -389,9 +389,9 @@ Expression ((Projection + (Before ORDER BY + (Projection + Before ORDER BY))))
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Before JOIN) Expression (Before JOIN)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -424,9 +424,9 @@ Expression (Projection)
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Before JOIN) Expression (Before JOIN)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
12 12
@ -455,9 +455,9 @@ Expression ((Projection + (Before ORDER BY + (Projection + Before ORDER BY))))
Expression ((Before GROUP BY + (Projection + Before ORDER BY))) Expression ((Before GROUP BY + (Projection + Before ORDER BY)))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Before JOIN) Expression (Before JOIN)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY))))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -474,7 +474,7 @@ Expression (Projection)
Expression (Before ORDER BY) Expression (Before ORDER BY)
Aggregating Aggregating
Expression (Before GROUP BY) Expression (Before GROUP BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
1 1
-- UNION ALL with DISTINCT => do _not_ remove DISTINCT -- UNION ALL with DISTINCT => do _not_ remove DISTINCT
@ -497,12 +497,12 @@ Expression (Projection)
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (( + Projection)) Expression (( + Projection))
Distinct Distinct
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression (Before ORDER BY) Expression (Before ORDER BY)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1

View File

@ -9,7 +9,7 @@ Expression (Project names)
Distinct (DISTINCT) Distinct (DISTINCT)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Projection + Change column names to column identifiers)) Expression ((Projection + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- Enabled query_plan_remove_redundant_distinct -- Enabled query_plan_remove_redundant_distinct
-- DISTINCT is only in most inner subquery -- DISTINCT is only in most inner subquery
-- query -- query
@ -28,7 +28,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
Distinct (DISTINCT) Distinct (DISTINCT)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Projection + Change column names to column identifiers)) Expression ((Projection + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -53,12 +53,12 @@ Expression (Project names)
Distinct (DISTINCT) Distinct (DISTINCT)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Projection + Change column names to column identifiers)) Expression ((Projection + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (( + ( + Project names))) Expression (( + ( + Project names)))
Distinct (DISTINCT) Distinct (DISTINCT)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Projection + Change column names to column identifiers)) Expression ((Projection + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1
@ -84,12 +84,12 @@ Expression (Project names)
Distinct (DISTINCT) Distinct (DISTINCT)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Projection + Change column names to column identifiers)) Expression ((Projection + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression ((Change column names to column identifiers + Project names)) Expression ((Change column names to column identifiers + Project names))
Distinct (DISTINCT) Distinct (DISTINCT)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Projection + Change column names to column identifiers)) Expression ((Projection + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0 0 0
0 1 0 1
@ -112,7 +112,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
Distinct (DISTINCT) Distinct (DISTINCT)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Projection + Change column names to column identifiers)) Expression ((Projection + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0 0 0
1 2 1 2
@ -134,7 +134,7 @@ Expression ((Project names + (Projection + (Change column names to column identi
Distinct (DISTINCT) Distinct (DISTINCT)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Projection + Change column names to column identifiers)) Expression ((Projection + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
2 0 0 2 0 0
2 1 2 2 1 2
@ -231,9 +231,9 @@ Expression ((Project names + (Projection + (Change column names to column identi
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -264,9 +264,9 @@ Expression (Project names)
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
12 12
-- GROUP BY WITH ROLLUP before DISTINCT with on different columns => do _not_ remove DISTINCT -- GROUP BY WITH ROLLUP before DISTINCT with on different columns => do _not_ remove DISTINCT
@ -296,9 +296,9 @@ Expression (Project names)
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
12 12
36 36
@ -326,9 +326,9 @@ Expression ((Project names + (Projection + (Change column names to column identi
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -361,9 +361,9 @@ Expression (Project names)
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
12 12
36 36
@ -391,9 +391,9 @@ Expression ((Project names + (Projection + (Change column names to column identi
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -426,9 +426,9 @@ Expression (Project names)
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
12 12
@ -457,9 +457,9 @@ Expression ((Project names + (Projection + (Change column names to column identi
Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN))))) Expression ((Before GROUP BY + (Change column names to column identifiers + (Project names + (Projection + DROP unused columns after JOIN)))))
Join (JOIN FillRightFirst) Join (JOIN FillRightFirst)
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (Change column names to column identifiers) Expression (Change column names to column identifiers)
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
2 2
@ -476,7 +476,7 @@ Expression (Project names)
Expression (Projection) Expression (Projection)
Aggregating Aggregating
Expression ((Before GROUP BY + Change column names to column identifiers)) Expression ((Before GROUP BY + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
1 1
-- UNION ALL with DISTINCT => do _not_ remove DISTINCT -- UNION ALL with DISTINCT => do _not_ remove DISTINCT
@ -499,12 +499,12 @@ Expression (Project names)
Distinct (DISTINCT) Distinct (DISTINCT)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Projection + Change column names to column identifiers)) Expression ((Projection + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Expression (( + ( + Project names))) Expression (( + ( + Project names)))
Distinct (DISTINCT) Distinct (DISTINCT)
Distinct (Preliminary DISTINCT) Distinct (Preliminary DISTINCT)
Expression ((Projection + Change column names to column identifiers)) Expression ((Projection + Change column names to column identifiers))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
-- execute -- execute
0 0
1 1

View File

@ -147,5 +147,5 @@ Header: bx String
c2 String c2 String
Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))) Expression ((JOIN actions + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers)))))
Header: d1_4 UInt64 Header: d1_4 UInt64
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Header: number UInt64 Header: number UInt64

View File

@ -18,5 +18,5 @@ Positions: 1
ALIAS number :: 0 -> number_1 UInt64 : 2 ALIAS number :: 0 -> number_1 UInt64 : 2
FUNCTION ignore(2_UInt8 :: 1) -> ignore(2_UInt8) UInt8 : 0 FUNCTION ignore(2_UInt8 :: 1) -> ignore(2_UInt8) UInt8 : 0
Positions: 0 2 Positions: 0 2
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers
Header: number UInt64 Header: number UInt64

View File

@ -1,2 +1,2 @@
Expression ((Projection + Before ORDER BY)) Expression ((Projection + Before ORDER BY))
ReadFromStorage (SystemNumbers) ReadFromSystemNumbers