Merge branch 'master' into vdimir/grouping_sets_aliases

This commit is contained in:
Vladimir Cherkasov 2024-11-15 16:58:11 +01:00 committed by GitHub
commit e106ae1232
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
69 changed files with 1710 additions and 1106 deletions

View File

@ -161,6 +161,8 @@ Settings:
- `actions` — Prints detailed information about step actions. Default: 0. - `actions` — Prints detailed information about step actions. Default: 0.
- `json` — Prints query plan steps as a row in [JSON](../../interfaces/formats.md#json) format. Default: 0. It is recommended to use [TSVRaw](../../interfaces/formats.md#tabseparatedraw) format to avoid unnecessary escaping. - `json` — Prints query plan steps as a row in [JSON](../../interfaces/formats.md#json) format. Default: 0. It is recommended to use [TSVRaw](../../interfaces/formats.md#tabseparatedraw) format to avoid unnecessary escaping.
When `json=1` step names will contain an additional suffix with unique step identifier.
Example: Example:
```sql ```sql
@ -194,30 +196,25 @@ EXPLAIN json = 1, description = 0 SELECT 1 UNION ALL SELECT 2 FORMAT TSVRaw;
{ {
"Plan": { "Plan": {
"Node Type": "Union", "Node Type": "Union",
"Node Id": "Union_10",
"Plans": [ "Plans": [
{ {
"Node Type": "Expression", "Node Type": "Expression",
"Node Id": "Expression_13",
"Plans": [ "Plans": [
{ {
"Node Type": "SettingQuotaAndLimits", "Node Type": "ReadFromStorage",
"Plans": [ "Node Id": "ReadFromStorage_0"
{
"Node Type": "ReadFromStorage"
}
]
} }
] ]
}, },
{ {
"Node Type": "Expression", "Node Type": "Expression",
"Node Id": "Expression_16",
"Plans": [ "Plans": [
{ {
"Node Type": "SettingQuotaAndLimits", "Node Type": "ReadFromStorage",
"Plans": [ "Node Id": "ReadFromStorage_4"
{
"Node Type": "ReadFromStorage"
}
]
} }
] ]
} }
@ -249,6 +246,7 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy;
{ {
"Plan": { "Plan": {
"Node Type": "Expression", "Node Type": "Expression",
"Node Id": "Expression_5",
"Header": [ "Header": [
{ {
"Name": "1", "Name": "1",
@ -261,23 +259,13 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy;
], ],
"Plans": [ "Plans": [
{ {
"Node Type": "SettingQuotaAndLimits", "Node Type": "ReadFromStorage",
"Node Id": "ReadFromStorage_0",
"Header": [ "Header": [
{ {
"Name": "dummy", "Name": "dummy",
"Type": "UInt8" "Type": "UInt8"
} }
],
"Plans": [
{
"Node Type": "ReadFromStorage",
"Header": [
{
"Name": "dummy",
"Type": "UInt8"
}
]
}
] ]
} }
] ]
@ -351,17 +339,31 @@ EXPLAIN json = 1, actions = 1, description = 0 SELECT 1 FORMAT TSVRaw;
{ {
"Plan": { "Plan": {
"Node Type": "Expression", "Node Type": "Expression",
"Node Id": "Expression_5",
"Expression": { "Expression": {
"Inputs": [], "Inputs": [
{
"Name": "dummy",
"Type": "UInt8"
}
],
"Actions": [ "Actions": [
{ {
"Node Type": "Column", "Node Type": "INPUT",
"Result Type": "UInt8", "Result Type": "UInt8",
"Result Type": "Column", "Result Name": "dummy",
"Arguments": [0],
"Removed Arguments": [0],
"Result": 0
},
{
"Node Type": "COLUMN",
"Result Type": "UInt8",
"Result Name": "1",
"Column": "Const(UInt8)", "Column": "Const(UInt8)",
"Arguments": [], "Arguments": [],
"Removed Arguments": [], "Removed Arguments": [],
"Result": 0 "Result": 1
} }
], ],
"Outputs": [ "Outputs": [
@ -370,17 +372,12 @@ EXPLAIN json = 1, actions = 1, description = 0 SELECT 1 FORMAT TSVRaw;
"Type": "UInt8" "Type": "UInt8"
} }
], ],
"Positions": [0], "Positions": [1]
"Project Input": true
}, },
"Plans": [ "Plans": [
{ {
"Node Type": "SettingQuotaAndLimits", "Node Type": "ReadFromStorage",
"Plans": [ "Node Id": "ReadFromStorage_0"
{
"Node Type": "ReadFromStorage"
}
]
} }
] ]
} }
@ -396,6 +393,8 @@ Settings:
- `graph` — Prints a graph described in the [DOT](https://en.wikipedia.org/wiki/DOT_(graph_description_language)) graph description language. Default: 0. - `graph` — Prints a graph described in the [DOT](https://en.wikipedia.org/wiki/DOT_(graph_description_language)) graph description language. Default: 0.
- `compact` — Prints graph in compact mode if `graph` setting is enabled. Default: 1. - `compact` — Prints graph in compact mode if `graph` setting is enabled. Default: 1.
When `compact=0` and `graph=1` processor names will contain an additional suffix with unique processor identifier.
Example: Example:
```sql ```sql

View File

@ -22,6 +22,13 @@ namespace ErrorCodes
namespace namespace
{ {
/** Due to a lack of proper code review, this code was contributed with a multiplication of template instantiations
* over all pairs of data types, and we deeply regret that.
*
* We cannot remove all combinations, because the binary representation of serialized data has to remain the same,
* but we can partially heal the wound by treating unsigned and signed data types in the same way.
*/
template <typename ValueType, typename TimestampType> template <typename ValueType, typename TimestampType>
struct AggregationFunctionDeltaSumTimestampData struct AggregationFunctionDeltaSumTimestampData
{ {
@ -37,23 +44,22 @@ template <typename ValueType, typename TimestampType>
class AggregationFunctionDeltaSumTimestamp final class AggregationFunctionDeltaSumTimestamp final
: public IAggregateFunctionDataHelper< : public IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>, AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType> AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>>
>
{ {
public: public:
AggregationFunctionDeltaSumTimestamp(const DataTypes & arguments, const Array & params) AggregationFunctionDeltaSumTimestamp(const DataTypes & arguments, const Array & params)
: IAggregateFunctionDataHelper< : IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>, AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType> AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>>{arguments, params, createResultType()}
>{arguments, params, createResultType()} {
{} }
AggregationFunctionDeltaSumTimestamp() AggregationFunctionDeltaSumTimestamp()
: IAggregateFunctionDataHelper< : IAggregateFunctionDataHelper<
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>, AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType> AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>>{}
>{} {
{} }
bool allocatesMemoryInArena() const override { return false; } bool allocatesMemoryInArena() const override { return false; }
@ -63,8 +69,8 @@ public:
void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override void NO_SANITIZE_UNDEFINED ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override
{ {
auto value = assert_cast<const ColumnVector<ValueType> &>(*columns[0]).getData()[row_num]; auto value = unalignedLoad<ValueType>(columns[0]->getRawData().data() + row_num * sizeof(ValueType));
auto ts = assert_cast<const ColumnVector<TimestampType> &>(*columns[1]).getData()[row_num]; auto ts = unalignedLoad<TimestampType>(columns[1]->getRawData().data() + row_num * sizeof(TimestampType));
auto & data = this->data(place); auto & data = this->data(place);
@ -172,10 +178,48 @@ public:
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{ {
assert_cast<ColumnVector<ValueType> &>(to).getData().push_back(this->data(place).sum); static_cast<ColumnFixedSizeHelper &>(to).template insertRawData<sizeof(ValueType)>(
reinterpret_cast<const char *>(&this->data(place).sum));
} }
}; };
template <typename FirstType, template <typename, typename> class AggregateFunctionTemplate, typename... TArgs>
IAggregateFunction * createWithTwoTypesSecond(const IDataType & second_type, TArgs && ... args)
{
WhichDataType which(second_type);
if (which.idx == TypeIndex::UInt32) return new AggregateFunctionTemplate<FirstType, UInt32>(args...);
if (which.idx == TypeIndex::UInt64) return new AggregateFunctionTemplate<FirstType, UInt64>(args...);
if (which.idx == TypeIndex::Int32) return new AggregateFunctionTemplate<FirstType, UInt32>(args...);
if (which.idx == TypeIndex::Int64) return new AggregateFunctionTemplate<FirstType, UInt64>(args...);
if (which.idx == TypeIndex::Float32) return new AggregateFunctionTemplate<FirstType, Float32>(args...);
if (which.idx == TypeIndex::Float64) return new AggregateFunctionTemplate<FirstType, Float64>(args...);
if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate<FirstType, UInt16>(args...);
if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate<FirstType, UInt32>(args...);
return nullptr;
}
template <template <typename, typename> class AggregateFunctionTemplate, typename... TArgs>
IAggregateFunction * createWithTwoTypes(const IDataType & first_type, const IDataType & second_type, TArgs && ... args)
{
WhichDataType which(first_type);
if (which.idx == TypeIndex::UInt8) return createWithTwoTypesSecond<UInt8, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::UInt16) return createWithTwoTypesSecond<UInt16, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::UInt32) return createWithTwoTypesSecond<UInt32, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::UInt64) return createWithTwoTypesSecond<UInt64, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::Int8) return createWithTwoTypesSecond<UInt8, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::Int16) return createWithTwoTypesSecond<UInt16, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::Int32) return createWithTwoTypesSecond<UInt32, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::Int64) return createWithTwoTypesSecond<UInt64, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::Float32) return createWithTwoTypesSecond<Float32, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::Float64) return createWithTwoTypesSecond<Float64, AggregateFunctionTemplate>(second_type, args...);
return nullptr;
}
AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp( AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
const String & name, const String & name,
const DataTypes & arguments, const DataTypes & arguments,
@ -193,8 +237,14 @@ AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}, " throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}, "
"must be Int, Float, Date, DateTime", arguments[1]->getName(), name); "must be Int, Float, Date, DateTime", arguments[1]->getName(), name);
return AggregateFunctionPtr(createWithTwoNumericOrDateTypes<AggregationFunctionDeltaSumTimestamp>( auto res = AggregateFunctionPtr(createWithTwoTypes<AggregationFunctionDeltaSumTimestamp>(
*arguments[0], *arguments[1], arguments, params)); *arguments[0], *arguments[1], arguments, params));
if (!res)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}, "
"this type is not supported", arguments[0]->getName(), name);
return res;
} }
} }

View File

@ -184,36 +184,8 @@ static IAggregateFunction * createWithDecimalType(const IDataType & argument_typ
} }
/** For template with two arguments. /** For template with two arguments.
* This is an extremely dangerous for code bloat - do not use.
*/ */
template <typename FirstType, template <typename, typename> class AggregateFunctionTemplate, typename... TArgs>
static IAggregateFunction * createWithTwoNumericTypesSecond(const IDataType & second_type, TArgs && ... args)
{
WhichDataType which(second_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate<FirstType, TYPE>(args...);
FOR_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate<FirstType, Int8>(args...);
if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate<FirstType, Int16>(args...);
return nullptr;
}
template <template <typename, typename> class AggregateFunctionTemplate, typename... TArgs>
static IAggregateFunction * createWithTwoNumericTypes(const IDataType & first_type, const IDataType & second_type, TArgs && ... args)
{
WhichDataType which(first_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) \
return createWithTwoNumericTypesSecond<TYPE, AggregateFunctionTemplate>(second_type, args...);
FOR_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
if (which.idx == TypeIndex::Enum8)
return createWithTwoNumericTypesSecond<Int8, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::Enum16)
return createWithTwoNumericTypesSecond<Int16, AggregateFunctionTemplate>(second_type, args...);
return nullptr;
}
template <typename FirstType, template <typename, typename> class AggregateFunctionTemplate, typename... TArgs> template <typename FirstType, template <typename, typename> class AggregateFunctionTemplate, typename... TArgs>
static IAggregateFunction * createWithTwoBasicNumericTypesSecond(const IDataType & second_type, TArgs && ... args) static IAggregateFunction * createWithTwoBasicNumericTypesSecond(const IDataType & second_type, TArgs && ... args)
{ {
@ -237,46 +209,6 @@ static IAggregateFunction * createWithTwoBasicNumericTypes(const IDataType & fir
return nullptr; return nullptr;
} }
template <typename FirstType, template <typename, typename> class AggregateFunctionTemplate, typename... TArgs>
static IAggregateFunction * createWithTwoNumericOrDateTypesSecond(const IDataType & second_type, TArgs && ... args)
{
WhichDataType which(second_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) return new AggregateFunctionTemplate<FirstType, TYPE>(args...);
FOR_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
if (which.idx == TypeIndex::Enum8) return new AggregateFunctionTemplate<FirstType, Int8>(args...);
if (which.idx == TypeIndex::Enum16) return new AggregateFunctionTemplate<FirstType, Int16>(args...);
/// expects that DataTypeDate based on UInt16, DataTypeDateTime based on UInt32
if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate<FirstType, UInt16>(args...);
if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate<FirstType, UInt32>(args...);
return nullptr;
}
template <template <typename, typename> class AggregateFunctionTemplate, typename... TArgs>
static IAggregateFunction * createWithTwoNumericOrDateTypes(const IDataType & first_type, const IDataType & second_type, TArgs && ... args)
{
WhichDataType which(first_type);
#define DISPATCH(TYPE) \
if (which.idx == TypeIndex::TYPE) \
return createWithTwoNumericOrDateTypesSecond<TYPE, AggregateFunctionTemplate>(second_type, args...);
FOR_NUMERIC_TYPES(DISPATCH)
#undef DISPATCH
if (which.idx == TypeIndex::Enum8)
return createWithTwoNumericOrDateTypesSecond<Int8, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::Enum16)
return createWithTwoNumericOrDateTypesSecond<Int16, AggregateFunctionTemplate>(second_type, args...);
/// expects that DataTypeDate based on UInt16, DataTypeDateTime based on UInt32
if (which.idx == TypeIndex::Date)
return createWithTwoNumericOrDateTypesSecond<UInt16, AggregateFunctionTemplate>(second_type, args...);
if (which.idx == TypeIndex::DateTime)
return createWithTwoNumericOrDateTypesSecond<UInt32, AggregateFunctionTemplate>(second_type, args...);
return nullptr;
}
template <template <typename> class AggregateFunctionTemplate, typename... TArgs> template <template <typename> class AggregateFunctionTemplate, typename... TArgs>
static IAggregateFunction * createWithStringType(const IDataType & argument_type, TArgs && ... args) static IAggregateFunction * createWithStringType(const IDataType & argument_type, TArgs && ... args)
{ {

View File

@ -49,6 +49,7 @@
M(TemporaryFilesForSort, "Number of temporary files created for external sorting") \ M(TemporaryFilesForSort, "Number of temporary files created for external sorting") \
M(TemporaryFilesForAggregation, "Number of temporary files created for external aggregation") \ M(TemporaryFilesForAggregation, "Number of temporary files created for external aggregation") \
M(TemporaryFilesForJoin, "Number of temporary files created for JOIN") \ M(TemporaryFilesForJoin, "Number of temporary files created for JOIN") \
M(TemporaryFilesForMerge, "Number of temporary files for vertical merge") \
M(TemporaryFilesUnknown, "Number of temporary files created without known purpose") \ M(TemporaryFilesUnknown, "Number of temporary files created without known purpose") \
M(Read, "Number of read (read, pread, io_getevents, etc.) syscalls in fly") \ M(Read, "Number of read (read, pread, io_getevents, etc.) syscalls in fly") \
M(RemoteRead, "Number of read with remote reader in fly") \ M(RemoteRead, "Number of read with remote reader in fly") \

View File

@ -204,6 +204,16 @@ bool ThreadStatus::isQueryCanceled() const
return false; return false;
} }
size_t ThreadStatus::getNextPlanStepIndex() const
{
return local_data.plan_step_index->fetch_add(1);
}
size_t ThreadStatus::getNextPipelineProcessorIndex() const
{
return local_data.pipeline_processor_index->fetch_add(1);
}
ThreadStatus::~ThreadStatus() ThreadStatus::~ThreadStatus()
{ {
flushUntrackedMemory(); flushUntrackedMemory();

View File

@ -11,6 +11,7 @@
#include <boost/noncopyable.hpp> #include <boost/noncopyable.hpp>
#include <atomic>
#include <functional> #include <functional>
#include <memory> #include <memory>
#include <mutex> #include <mutex>
@ -90,6 +91,11 @@ public:
String query_for_logs; String query_for_logs;
UInt64 normalized_query_hash = 0; UInt64 normalized_query_hash = 0;
// Since processors might be added on the fly within expand() function we use atomic_size_t.
// These two fields are used for EXPLAIN PLAN / PIPELINE.
std::shared_ptr<std::atomic_size_t> plan_step_index = std::make_shared<std::atomic_size_t>(0);
std::shared_ptr<std::atomic_size_t> pipeline_processor_index = std::make_shared<std::atomic_size_t>(0);
QueryIsCanceledPredicate query_is_canceled_predicate = {}; QueryIsCanceledPredicate query_is_canceled_predicate = {};
}; };
@ -313,6 +319,9 @@ public:
void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period); void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period);
size_t getNextPlanStepIndex() const;
size_t getNextPipelineProcessorIndex() const;
private: private:
void applyGlobalSettings(); void applyGlobalSettings();
void applyQuerySettings(); void applyQuerySettings();

View File

@ -4565,7 +4565,7 @@ Possible values:
- 0 - Disable - 0 - Disable
- 1 - Enable - 1 - Enable
)", 0) \ )", 0) \
DECLARE(Bool, query_plan_merge_filters, false, R"( DECLARE(Bool, query_plan_merge_filters, true, R"(
Allow to merge filters in the query plan Allow to merge filters in the query plan
)", 0) \ )", 0) \
DECLARE(Bool, query_plan_filter_push_down, true, R"( DECLARE(Bool, query_plan_filter_push_down, true, R"(

View File

@ -77,6 +77,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"backup_restore_keeper_max_retries_while_initializing", 0, 20, "New setting."}, {"backup_restore_keeper_max_retries_while_initializing", 0, 20, "New setting."},
{"backup_restore_keeper_max_retries_while_handling_error", 0, 20, "New setting."}, {"backup_restore_keeper_max_retries_while_handling_error", 0, 20, "New setting."},
{"backup_restore_finish_timeout_after_error_sec", 0, 180, "New setting."}, {"backup_restore_finish_timeout_after_error_sec", 0, 180, "New setting."},
{"query_plan_merge_filters", false, true, "Allow to merge filters in the query plan. This is required to properly support filter-push-down with a new analyzer."},
{"parallel_replicas_local_plan", false, true, "Use local plan for local replica in a query with parallel replicas"}, {"parallel_replicas_local_plan", false, true, "Use local plan for local replica in a query with parallel replicas"},
{"allow_experimental_bfloat16_type", false, false, "Add new experimental BFloat16 type"}, {"allow_experimental_bfloat16_type", false, false, "Add new experimental BFloat16 type"},
{"filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit", 1, 1, "Rename of setting skip_download_if_exceeds_query_cache_limit"}, {"filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit", 1, 1, "Rename of setting skip_download_if_exceeds_query_cache_limit"},

View File

@ -69,7 +69,7 @@ static void testCascadeBufferRedability(
auto rbuf = wbuf_readable.tryGetReadBuffer(); auto rbuf = wbuf_readable.tryGetReadBuffer();
ASSERT_FALSE(!rbuf); ASSERT_FALSE(!rbuf);
concat.appendBuffer(wrapReadBufferPointer(std::move(rbuf))); concat.appendBuffer(std::move(rbuf));
} }
std::string decoded_data; std::string decoded_data;

View File

@ -335,7 +335,7 @@ Aggregator::Aggregator(const Block & header_, const Params & params_)
: header(header_) : header(header_)
, keys_positions(calculateKeysPositions(header, params_)) , keys_positions(calculateKeysPositions(header, params_))
, params(params_) , params(params_)
, tmp_data(params.tmp_data_scope ? std::make_unique<TemporaryDataOnDisk>(params.tmp_data_scope, CurrentMetrics::TemporaryFilesForAggregation) : nullptr) , tmp_data(params.tmp_data_scope ? params.tmp_data_scope->childScope(CurrentMetrics::TemporaryFilesForAggregation) : nullptr)
, min_bytes_for_prefetch(getMinBytesForPrefetch()) , min_bytes_for_prefetch(getMinBytesForPrefetch())
{ {
/// Use query-level memory tracker /// Use query-level memory tracker
@ -1519,10 +1519,15 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, si
Stopwatch watch; Stopwatch watch;
size_t rows = data_variants.size(); size_t rows = data_variants.size();
auto & out_stream = tmp_data->createStream(getHeader(false), max_temp_file_size); auto & out_stream = [this, max_temp_file_size]() -> TemporaryBlockStreamHolder &
{
std::lock_guard lk(tmp_files_mutex);
return tmp_files.emplace_back(getHeader(false), tmp_data.get(), max_temp_file_size);
}();
ProfileEvents::increment(ProfileEvents::ExternalAggregationWritePart); ProfileEvents::increment(ProfileEvents::ExternalAggregationWritePart);
LOG_DEBUG(log, "Writing part of aggregation data into temporary file {}", out_stream.getPath()); LOG_DEBUG(log, "Writing part of aggregation data into temporary file {}", out_stream.getHolder()->describeFilePath());
/// Flush only two-level data and possibly overflow data. /// Flush only two-level data and possibly overflow data.
@ -1639,11 +1644,24 @@ Block Aggregator::convertOneBucketToBlock(AggregatedDataVariants & variants, Are
return block; return block;
} }
std::list<TemporaryBlockStreamHolder> Aggregator::detachTemporaryData()
{
std::lock_guard lk(tmp_files_mutex);
return std::move(tmp_files);
}
bool Aggregator::hasTemporaryData() const
{
std::lock_guard lk(tmp_files_mutex);
return !tmp_files.empty();
}
template <typename Method> template <typename Method>
void Aggregator::writeToTemporaryFileImpl( void Aggregator::writeToTemporaryFileImpl(
AggregatedDataVariants & data_variants, AggregatedDataVariants & data_variants,
Method & method, Method & method,
TemporaryFileStream & out) const TemporaryBlockStreamHolder & out) const
{ {
size_t max_temporary_block_size_rows = 0; size_t max_temporary_block_size_rows = 0;
size_t max_temporary_block_size_bytes = 0; size_t max_temporary_block_size_bytes = 0;
@ -1660,14 +1678,14 @@ void Aggregator::writeToTemporaryFileImpl(
for (UInt32 bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket) for (UInt32 bucket = 0; bucket < Method::Data::NUM_BUCKETS; ++bucket)
{ {
Block block = convertOneBucketToBlock(data_variants, method, data_variants.aggregates_pool, false, bucket); Block block = convertOneBucketToBlock(data_variants, method, data_variants.aggregates_pool, false, bucket);
out.write(block); out->write(block);
update_max_sizes(block); update_max_sizes(block);
} }
if (params.overflow_row) if (params.overflow_row)
{ {
Block block = prepareBlockAndFillWithoutKey(data_variants, false, true); Block block = prepareBlockAndFillWithoutKey(data_variants, false, true);
out.write(block); out->write(block);
update_max_sizes(block); update_max_sizes(block);
} }

View File

@ -309,9 +309,9 @@ public:
/// For external aggregation. /// For external aggregation.
void writeToTemporaryFile(AggregatedDataVariants & data_variants, size_t max_temp_file_size = 0) const; void writeToTemporaryFile(AggregatedDataVariants & data_variants, size_t max_temp_file_size = 0) const;
bool hasTemporaryData() const { return tmp_data && !tmp_data->empty(); } bool hasTemporaryData() const;
const TemporaryDataOnDisk & getTemporaryData() const { return *tmp_data; } std::list<TemporaryBlockStreamHolder> detachTemporaryData();
/// Get data structure of the result. /// Get data structure of the result.
Block getHeader(bool final) const; Block getHeader(bool final) const;
@ -355,7 +355,9 @@ private:
LoggerPtr log = getLogger("Aggregator"); LoggerPtr log = getLogger("Aggregator");
/// For external aggregation. /// For external aggregation.
TemporaryDataOnDiskPtr tmp_data; TemporaryDataOnDiskScopePtr tmp_data;
mutable std::mutex tmp_files_mutex;
mutable std::list<TemporaryBlockStreamHolder> tmp_files TSA_GUARDED_BY(tmp_files_mutex);
size_t min_bytes_for_prefetch = 0; size_t min_bytes_for_prefetch = 0;
@ -456,7 +458,7 @@ private:
void writeToTemporaryFileImpl( void writeToTemporaryFileImpl(
AggregatedDataVariants & data_variants, AggregatedDataVariants & data_variants,
Method & method, Method & method,
TemporaryFileStream & out) const; TemporaryBlockStreamHolder & out) const;
/// Merge NULL key data from hash table `src` into `dst`. /// Merge NULL key data from hash table `src` into `dst`.
template <typename Method, typename Table> template <typename Method, typename Table>

View File

@ -364,6 +364,8 @@ struct ContextSharedPart : boost::noncopyable
/// Child scopes for more fine-grained accounting are created per user/query/etc. /// Child scopes for more fine-grained accounting are created per user/query/etc.
/// Initialized once during server startup. /// Initialized once during server startup.
TemporaryDataOnDiskScopePtr root_temp_data_on_disk TSA_GUARDED_BY(mutex); TemporaryDataOnDiskScopePtr root_temp_data_on_disk TSA_GUARDED_BY(mutex);
/// TODO: remove, use only root_temp_data_on_disk
VolumePtr temporary_volume_legacy;
mutable OnceFlag async_loader_initialized; mutable OnceFlag async_loader_initialized;
mutable std::unique_ptr<AsyncLoader> async_loader; /// Thread pool for asynchronous initialization of arbitrary DAG of `LoadJob`s (used for tables loading) mutable std::unique_ptr<AsyncLoader> async_loader; /// Thread pool for asynchronous initialization of arbitrary DAG of `LoadJob`s (used for tables loading)
@ -799,10 +801,9 @@ struct ContextSharedPart : boost::noncopyable
} }
/// Special volumes might also use disks that require shutdown. /// Special volumes might also use disks that require shutdown.
auto & tmp_data = root_temp_data_on_disk; if (temporary_volume_legacy)
if (tmp_data && tmp_data->getVolume())
{ {
auto & disks = tmp_data->getVolume()->getDisks(); auto & disks = temporary_volume_legacy->getDisks();
for (auto & disk : disks) for (auto & disk : disks)
disk->shutdown(); disk->shutdown();
} }
@ -1184,8 +1185,8 @@ VolumePtr Context::getGlobalTemporaryVolume() const
SharedLockGuard lock(shared->mutex); SharedLockGuard lock(shared->mutex);
/// Calling this method we just bypass the `temp_data_on_disk` and write to the file on the volume directly. /// Calling this method we just bypass the `temp_data_on_disk` and write to the file on the volume directly.
/// Volume is the same for `root_temp_data_on_disk` (always set) and `temp_data_on_disk` (if it's set). /// Volume is the same for `root_temp_data_on_disk` (always set) and `temp_data_on_disk` (if it's set).
if (shared->root_temp_data_on_disk) if (shared->temporary_volume_legacy)
return shared->root_temp_data_on_disk->getVolume(); return shared->temporary_volume_legacy;
return nullptr; return nullptr;
} }
@ -1273,6 +1274,10 @@ try
/// We skip directories (for example, 'http_buffers' - it's used for buffering of the results) and all other file types. /// We skip directories (for example, 'http_buffers' - it's used for buffering of the results) and all other file types.
} }
} }
else
{
fs::create_directories(path);
}
} }
catch (...) catch (...)
{ {
@ -1306,7 +1311,8 @@ void Context::setTemporaryStoragePath(const String & path, size_t max_size)
TemporaryDataOnDiskSettings temporary_data_on_disk_settings; TemporaryDataOnDiskSettings temporary_data_on_disk_settings;
temporary_data_on_disk_settings.max_size_on_disk = max_size; temporary_data_on_disk_settings.max_size_on_disk = max_size;
shared->root_temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(std::move(volume), std::move(temporary_data_on_disk_settings)); shared->root_temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(volume, std::move(temporary_data_on_disk_settings));
shared->temporary_volume_legacy = volume;
} }
void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_size) void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_size)
@ -1354,7 +1360,8 @@ void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_s
TemporaryDataOnDiskSettings temporary_data_on_disk_settings; TemporaryDataOnDiskSettings temporary_data_on_disk_settings;
temporary_data_on_disk_settings.max_size_on_disk = max_size; temporary_data_on_disk_settings.max_size_on_disk = max_size;
shared->root_temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(std::move(volume), std::move(temporary_data_on_disk_settings)); shared->root_temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(volume, std::move(temporary_data_on_disk_settings));
shared->temporary_volume_legacy = volume;
} }
void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size) void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t max_size)
@ -1378,7 +1385,8 @@ void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t
TemporaryDataOnDiskSettings temporary_data_on_disk_settings; TemporaryDataOnDiskSettings temporary_data_on_disk_settings;
temporary_data_on_disk_settings.max_size_on_disk = max_size; temporary_data_on_disk_settings.max_size_on_disk = max_size;
shared->root_temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(std::move(volume), file_cache.get(), std::move(temporary_data_on_disk_settings)); shared->root_temp_data_on_disk = std::make_shared<TemporaryDataOnDiskScope>(file_cache.get(), std::move(temporary_data_on_disk_settings));
shared->temporary_volume_legacy = volume;
} }
void Context::setFlagsPath(const String & path) void Context::setFlagsPath(const String & path)

View File

@ -41,15 +41,15 @@ namespace
class AccumulatedBlockReader class AccumulatedBlockReader
{ {
public: public:
AccumulatedBlockReader(TemporaryFileStream & reader_, AccumulatedBlockReader(TemporaryBlockStreamReaderHolder reader_,
std::mutex & mutex_, std::mutex & mutex_,
size_t result_block_size_ = 0) size_t result_block_size_ = 0)
: reader(reader_) : reader(std::move(reader_))
, mutex(mutex_) , mutex(mutex_)
, result_block_size(result_block_size_) , result_block_size(result_block_size_)
{ {
if (!reader.isWriteFinished()) if (!reader)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Reading not finished file"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Reader is nullptr");
} }
Block read() Block read()
@ -63,7 +63,7 @@ namespace
size_t rows_read = 0; size_t rows_read = 0;
do do
{ {
Block block = reader.read(); Block block = reader->read();
rows_read += block.rows(); rows_read += block.rows();
if (!block) if (!block)
{ {
@ -81,7 +81,7 @@ namespace
} }
private: private:
TemporaryFileStream & reader; TemporaryBlockStreamReaderHolder reader;
std::mutex & mutex; std::mutex & mutex;
const size_t result_block_size; const size_t result_block_size;
@ -124,12 +124,12 @@ class GraceHashJoin::FileBucket : boost::noncopyable
public: public:
using BucketLock = std::unique_lock<std::mutex>; using BucketLock = std::unique_lock<std::mutex>;
explicit FileBucket(size_t bucket_index_, TemporaryFileStream & left_file_, TemporaryFileStream & right_file_, LoggerPtr log_) explicit FileBucket(size_t bucket_index_, TemporaryBlockStreamHolder left_file_, TemporaryBlockStreamHolder right_file_, LoggerPtr log_)
: idx{bucket_index_} : idx(bucket_index_)
, left_file{left_file_} , left_file(std::move(left_file_))
, right_file{right_file_} , right_file(std::move(right_file_))
, state{State::WRITING_BLOCKS} , state(State::WRITING_BLOCKS)
, log{log_} , log(log_)
{ {
} }
@ -157,12 +157,6 @@ public:
return addBlockImpl(block, right_file, lock); return addBlockImpl(block, right_file, lock);
} }
bool finished() const
{
std::unique_lock<std::mutex> left_lock(left_file_mutex);
return left_file.isEof();
}
bool empty() const { return is_empty.load(); } bool empty() const { return is_empty.load(); }
AccumulatedBlockReader startJoining() AccumulatedBlockReader startJoining()
@ -172,24 +166,21 @@ public:
std::unique_lock<std::mutex> left_lock(left_file_mutex); std::unique_lock<std::mutex> left_lock(left_file_mutex);
std::unique_lock<std::mutex> right_lock(right_file_mutex); std::unique_lock<std::mutex> right_lock(right_file_mutex);
left_file.finishWriting();
right_file.finishWriting();
state = State::JOINING_BLOCKS; state = State::JOINING_BLOCKS;
} }
return AccumulatedBlockReader(right_file, right_file_mutex); return AccumulatedBlockReader(right_file.getReadStream(), right_file_mutex);
} }
AccumulatedBlockReader getLeftTableReader() AccumulatedBlockReader getLeftTableReader()
{ {
ensureState(State::JOINING_BLOCKS); ensureState(State::JOINING_BLOCKS);
return AccumulatedBlockReader(left_file, left_file_mutex); return AccumulatedBlockReader(left_file.getReadStream(), left_file_mutex);
} }
const size_t idx; const size_t idx;
private: private:
bool addBlockImpl(const Block & block, TemporaryFileStream & writer, std::unique_lock<std::mutex> & lock) bool addBlockImpl(const Block & block, TemporaryBlockStreamHolder & writer, std::unique_lock<std::mutex> & lock)
{ {
ensureState(State::WRITING_BLOCKS); ensureState(State::WRITING_BLOCKS);
@ -199,7 +190,7 @@ private:
if (block.rows()) if (block.rows())
is_empty = false; is_empty = false;
writer.write(block); writer->write(block);
return true; return true;
} }
@ -217,8 +208,8 @@ private:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid state transition, expected {}, got {}", expected, state.load()); throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid state transition, expected {}, got {}", expected, state.load());
} }
TemporaryFileStream & left_file; TemporaryBlockStreamHolder left_file;
TemporaryFileStream & right_file; TemporaryBlockStreamHolder right_file;
mutable std::mutex left_file_mutex; mutable std::mutex left_file_mutex;
mutable std::mutex right_file_mutex; mutable std::mutex right_file_mutex;
@ -274,7 +265,7 @@ GraceHashJoin::GraceHashJoin(
, max_num_buckets{context->getSettingsRef()[Setting::grace_hash_join_max_buckets]} , max_num_buckets{context->getSettingsRef()[Setting::grace_hash_join_max_buckets]}
, left_key_names(table_join->getOnlyClause().key_names_left) , left_key_names(table_join->getOnlyClause().key_names_left)
, right_key_names(table_join->getOnlyClause().key_names_right) , right_key_names(table_join->getOnlyClause().key_names_right)
, tmp_data(std::make_unique<TemporaryDataOnDisk>(tmp_data_, CurrentMetrics::TemporaryFilesForJoin)) , tmp_data(tmp_data_->childScope(CurrentMetrics::TemporaryFilesForJoin))
, hash_join(makeInMemoryJoin("grace0")) , hash_join(makeInMemoryJoin("grace0"))
, hash_join_sample_block(hash_join->savedBlockSample()) , hash_join_sample_block(hash_join->savedBlockSample())
{ {
@ -398,10 +389,10 @@ void GraceHashJoin::addBuckets(const size_t bucket_count)
for (size_t i = 0; i < bucket_count; ++i) for (size_t i = 0; i < bucket_count; ++i)
try try
{ {
auto & left_file = tmp_data->createStream(left_sample_block); TemporaryBlockStreamHolder left_file(left_sample_block, tmp_data.get());
auto & right_file = tmp_data->createStream(prepareRightBlock(right_sample_block)); TemporaryBlockStreamHolder right_file(prepareRightBlock(right_sample_block), tmp_data.get());
BucketPtr new_bucket = std::make_shared<FileBucket>(current_size + i, left_file, right_file, log); BucketPtr new_bucket = std::make_shared<FileBucket>(current_size + i, std::move(left_file), std::move(right_file), log);
tmp_buckets.emplace_back(std::move(new_bucket)); tmp_buckets.emplace_back(std::move(new_bucket));
} }
catch (...) catch (...)
@ -632,12 +623,9 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks()
for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx) for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx)
{ {
current_bucket = buckets[bucket_idx].get(); current_bucket = buckets[bucket_idx].get();
if (current_bucket->finished() || current_bucket->empty()) if (current_bucket->empty())
{ {
LOG_TRACE(log, "Skipping {} {} bucket {}", LOG_TRACE(log, "Skipping empty bucket {}", bucket_idx);
current_bucket->finished() ? "finished" : "",
current_bucket->empty() ? "empty" : "",
bucket_idx);
continue; continue;
} }

View File

@ -132,7 +132,7 @@ private:
Names left_key_names; Names left_key_names;
Names right_key_names; Names right_key_names;
TemporaryDataOnDiskPtr tmp_data; TemporaryDataOnDiskScopePtr tmp_data;
Buckets buckets; Buckets buckets;
mutable SharedMutex rehash_mutex; mutable SharedMutex rehash_mutex;

View File

@ -35,11 +35,6 @@
#include <Interpreters/HashJoin/HashJoinMethods.h> #include <Interpreters/HashJoin/HashJoinMethods.h>
#include <Interpreters/HashJoin/JoinUsedFlags.h> #include <Interpreters/HashJoin/JoinUsedFlags.h>
namespace CurrentMetrics
{
extern const Metric TemporaryFilesForJoin;
}
namespace DB namespace DB
{ {
@ -64,7 +59,7 @@ struct NotProcessedCrossJoin : public ExtraBlock
{ {
size_t left_position; size_t left_position;
size_t right_block; size_t right_block;
std::unique_ptr<TemporaryFileStream::Reader> reader; std::optional<TemporaryBlockStreamReaderHolder> reader;
}; };
@ -106,10 +101,7 @@ HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_s
, instance_id(instance_id_) , instance_id(instance_id_)
, asof_inequality(table_join->getAsofInequality()) , asof_inequality(table_join->getAsofInequality())
, data(std::make_shared<RightTableData>()) , data(std::make_shared<RightTableData>())
, tmp_data( , tmp_data(table_join_->getTempDataOnDisk())
table_join_->getTempDataOnDisk()
? std::make_unique<TemporaryDataOnDisk>(table_join_->getTempDataOnDisk(), CurrentMetrics::TemporaryFilesForJoin)
: nullptr)
, right_sample_block(right_sample_block_) , right_sample_block(right_sample_block_)
, max_joined_block_rows(table_join->maxJoinedBlockRows()) , max_joined_block_rows(table_join->maxJoinedBlockRows())
, instance_log_id(!instance_id_.empty() ? "(" + instance_id_ + ") " : "") , instance_log_id(!instance_id_.empty() ? "(" + instance_id_ + ") " : "")
@ -520,11 +512,10 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
&& (tmp_stream || (max_bytes_in_join && getTotalByteCount() + block_to_save.allocatedBytes() >= max_bytes_in_join) && (tmp_stream || (max_bytes_in_join && getTotalByteCount() + block_to_save.allocatedBytes() >= max_bytes_in_join)
|| (max_rows_in_join && getTotalRowCount() + block_to_save.rows() >= max_rows_in_join))) || (max_rows_in_join && getTotalRowCount() + block_to_save.rows() >= max_rows_in_join)))
{ {
if (tmp_stream == nullptr) if (!tmp_stream)
{ tmp_stream.emplace(right_sample_block, tmp_data.get());
tmp_stream = &tmp_data->createStream(right_sample_block);
} tmp_stream.value()->write(block_to_save);
tmp_stream->write(block_to_save);
return true; return true;
} }
@ -730,13 +721,14 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed)
{ {
size_t start_left_row = 0; size_t start_left_row = 0;
size_t start_right_block = 0; size_t start_right_block = 0;
std::unique_ptr<TemporaryFileStream::Reader> reader = nullptr; std::optional<TemporaryBlockStreamReaderHolder> reader;
if (not_processed) if (not_processed)
{ {
auto & continuation = static_cast<NotProcessedCrossJoin &>(*not_processed); auto & continuation = static_cast<NotProcessedCrossJoin &>(*not_processed);
start_left_row = continuation.left_position; start_left_row = continuation.left_position;
start_right_block = continuation.right_block; start_right_block = continuation.right_block;
reader = std::move(continuation.reader); if (continuation.reader)
reader = std::move(*continuation.reader);
not_processed.reset(); not_processed.reset();
} }
@ -804,12 +796,10 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed)
if (tmp_stream && rows_added <= max_joined_block_rows) if (tmp_stream && rows_added <= max_joined_block_rows)
{ {
if (reader == nullptr) if (!reader)
{
tmp_stream->finishWritingAsyncSafe();
reader = tmp_stream->getReadStream(); reader = tmp_stream->getReadStream();
}
while (auto block_right = reader->read()) while (auto block_right = reader.value()->read())
{ {
++block_number; ++block_number;
process_right_block(block_right); process_right_block(block_right);

View File

@ -423,8 +423,9 @@ private:
std::vector<Sizes> key_sizes; std::vector<Sizes> key_sizes;
/// Needed to do external cross join /// Needed to do external cross join
TemporaryDataOnDiskPtr tmp_data; TemporaryDataOnDiskScopePtr tmp_data;
TemporaryFileStream* tmp_stream{nullptr}; std::optional<TemporaryBlockStreamHolder> tmp_stream;
mutable std::once_flag finish_writing;
/// Block with columns from the right-side table. /// Block with columns from the right-side table.
Block right_sample_block; Block right_sample_block;

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include <memory> #include <memory>
#include <vector>
#include <Core/Names.h> #include <Core/Names.h>
#include <Core/Block.h> #include <Core/Block.h>

View File

@ -48,6 +48,8 @@ ColumnsDescription ProcessorProfileLogElement::getColumnsDescription()
{"input_bytes", std::make_shared<DataTypeUInt64>(), "The number of bytes consumed by processor."}, {"input_bytes", std::make_shared<DataTypeUInt64>(), "The number of bytes consumed by processor."},
{"output_rows", std::make_shared<DataTypeUInt64>(), "The number of rows generated by processor."}, {"output_rows", std::make_shared<DataTypeUInt64>(), "The number of rows generated by processor."},
{"output_bytes", std::make_shared<DataTypeUInt64>(), "The number of bytes generated by processor."}, {"output_bytes", std::make_shared<DataTypeUInt64>(), "The number of bytes generated by processor."},
{"processor_uniq_id", std::make_shared<DataTypeString>(), "The uniq processor id in pipeline."},
{"step_uniq_id", std::make_shared<DataTypeString>(), "The uniq step id in plan."},
}; };
} }
@ -83,6 +85,8 @@ void ProcessorProfileLogElement::appendToBlock(MutableColumns & columns) const
columns[i++]->insert(input_bytes); columns[i++]->insert(input_bytes);
columns[i++]->insert(output_rows); columns[i++]->insert(output_rows);
columns[i++]->insert(output_bytes); columns[i++]->insert(output_bytes);
columns[i++]->insert(processor_uniq_id);
columns[i++]->insert(step_uniq_id);
} }
void logProcessorProfile(ContextPtr context, const Processors & processors) void logProcessorProfile(ContextPtr context, const Processors & processors)
@ -120,6 +124,8 @@ void logProcessorProfile(ContextPtr context, const Processors & processors)
processor_elem.plan_step_name = processor->getPlanStepName(); processor_elem.plan_step_name = processor->getPlanStepName();
processor_elem.plan_step_description = processor->getPlanStepDescription(); processor_elem.plan_step_description = processor->getPlanStepDescription();
processor_elem.plan_group = processor->getQueryPlanStepGroup(); processor_elem.plan_group = processor->getQueryPlanStepGroup();
processor_elem.processor_uniq_id = processor->getUniqID();
processor_elem.step_uniq_id = processor->getStepUniqID();
processor_elem.processor_name = processor->getName(); processor_elem.processor_name = processor->getName();

View File

@ -17,7 +17,7 @@ struct ProcessorProfileLogElement
UInt64 id{}; UInt64 id{};
std::vector<UInt64> parent_ids; std::vector<UInt64> parent_ids;
UInt64 plan_step{}; UInt64 plan_step;
UInt64 plan_group{}; UInt64 plan_group{};
String plan_step_name; String plan_step_name;
String plan_step_description; String plan_step_description;
@ -25,6 +25,8 @@ struct ProcessorProfileLogElement
String initial_query_id; String initial_query_id;
String query_id; String query_id;
String processor_name; String processor_name;
String processor_uniq_id;
String step_uniq_id;
/// Milliseconds spend in IProcessor::work() /// Milliseconds spend in IProcessor::work()
UInt64 elapsed_us{}; UInt64 elapsed_us{};

View File

@ -20,6 +20,11 @@
#include <memory> #include <memory>
#include <base/types.h> #include <base/types.h>
namespace CurrentMetrics
{
extern const Metric TemporaryFilesForJoin;
}
namespace DB namespace DB
{ {
@ -265,7 +270,7 @@ public:
VolumePtr getGlobalTemporaryVolume() { return tmp_volume; } VolumePtr getGlobalTemporaryVolume() { return tmp_volume; }
TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data; } TemporaryDataOnDiskScopePtr getTempDataOnDisk() { return tmp_data ? tmp_data->childScope(CurrentMetrics::TemporaryFilesForJoin) : nullptr; }
ActionsDAG createJoinedBlockActions(ContextPtr context) const; ActionsDAG createJoinedBlockActions(ContextPtr context) const;

View File

@ -9,13 +9,16 @@
#include <Interpreters/Cache/FileCache.h> #include <Interpreters/Cache/FileCache.h>
#include <Formats/NativeWriter.h> #include <Formats/NativeWriter.h>
#include <Core/ProtocolDefines.h> #include <Core/ProtocolDefines.h>
#include <Disks/IDisk.h>
#include <Disks/SingleDiskVolume.h> #include <Disks/SingleDiskVolume.h>
#include <Disks/DiskLocal.h> #include <Disks/DiskLocal.h>
#include <Disks/IO/WriteBufferFromTemporaryFile.h> #include <Disks/IO/WriteBufferFromTemporaryFile.h>
#include <Core/Defines.h> #include <Core/Defines.h>
#include <Common/formatReadable.h>
#include <Common/NaNUtils.h>
#include <Interpreters/Cache/WriteBufferToFileSegment.h> #include <Interpreters/Cache/WriteBufferToFileSegment.h>
#include "Common/Exception.h" #include <Common/Exception.h>
namespace ProfileEvents namespace ProfileEvents
{ {
@ -27,11 +30,293 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int TOO_MANY_ROWS_OR_BYTES; extern const int INVALID_STATE;
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int NOT_ENOUGH_SPACE; extern const int NOT_ENOUGH_SPACE;
extern const int TOO_MANY_ROWS_OR_BYTES;
} }
namespace
{
inline CompressionCodecPtr getCodec(const TemporaryDataOnDiskSettings & settings)
{
if (settings.compression_codec.empty())
return CompressionCodecFactory::instance().get("NONE");
return CompressionCodecFactory::instance().get(settings.compression_codec);
}
}
TemporaryFileHolder::TemporaryFileHolder()
{
ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal);
}
class TemporaryFileInLocalCache : public TemporaryFileHolder
{
public:
explicit TemporaryFileInLocalCache(FileCache & file_cache, size_t reserve_size = 0)
{
const auto key = FileSegment::Key::random();
LOG_TRACE(getLogger("TemporaryFileInLocalCache"), "Creating temporary file in cache with key {}", key);
segment_holder = file_cache.set(
key, 0, std::max<size_t>(1, reserve_size),
CreateFileSegmentSettings(FileSegmentKind::Ephemeral), FileCache::getCommonUser());
chassert(segment_holder->size() == 1);
segment_holder->front().getKeyMetadata()->createBaseDirectory(/* throw_if_failed */true);
}
std::unique_ptr<WriteBuffer> write() override
{
return std::make_unique<WriteBufferToFileSegment>(&segment_holder->front());
}
std::unique_ptr<ReadBuffer> read(size_t buffer_size) const override
{
return std::make_unique<ReadBufferFromFile>(segment_holder->front().getPath(), /* buf_size = */ buffer_size);
}
String describeFilePath() const override
{
return fmt::format("fscache://{}", segment_holder->front().getPath());
}
private:
FileSegmentsHolderPtr segment_holder;
};
class TemporaryFileOnLocalDisk : public TemporaryFileHolder
{
public:
explicit TemporaryFileOnLocalDisk(VolumePtr volume, size_t reserve_size = 0)
: path_to_file("tmp" + toString(UUIDHelpers::generateV4()))
{
LOG_TRACE(getLogger("TemporaryFileOnLocalDisk"), "Creating temporary file '{}'", path_to_file);
if (reserve_size > 0)
{
auto reservation = volume->reserve(reserve_size);
if (!reservation)
{
auto disks = volume->getDisks();
Strings disks_info;
for (const auto & d : disks)
{
auto to_double = [](auto x) { return static_cast<double>(x); };
disks_info.push_back(fmt::format("{}: available: {} unreserved: {}, total: {}, keeping: {}",
d->getName(),
ReadableSize(d->getAvailableSpace().transform(to_double).value_or(NaNOrZero<double>())),
ReadableSize(d->getUnreservedSpace().transform(to_double).value_or(NaNOrZero<double>())),
ReadableSize(d->getTotalSpace().transform(to_double).value_or(NaNOrZero<double>())),
ReadableSize(d->getKeepingFreeSpace())));
}
throw Exception(ErrorCodes::NOT_ENOUGH_SPACE,
"Not enough space on temporary disk, cannot reserve {} bytes on [{}]",
reserve_size, fmt::join(disks_info, ", "));
}
disk = reservation->getDisk();
}
else
{
disk = volume->getDisk();
}
chassert(disk);
}
std::unique_ptr<WriteBuffer> write() override
{
return disk->writeFile(path_to_file);
}
std::unique_ptr<ReadBuffer> read(size_t buffer_size) const override
{
ReadSettings settings;
settings.local_fs_buffer_size = buffer_size;
settings.remote_fs_buffer_size = buffer_size;
settings.prefetch_buffer_size = buffer_size;
return disk->readFile(path_to_file, settings);
}
String describeFilePath() const override
{
return fmt::format("disk({})://{}/{}", disk->getName(), disk->getPath(), path_to_file);
}
~TemporaryFileOnLocalDisk() override
try
{
if (disk->existsFile(path_to_file))
{
LOG_TRACE(getLogger("TemporaryFileOnLocalDisk"), "Removing temporary file '{}'", path_to_file);
disk->removeRecursive(path_to_file);
}
else
{
LOG_WARNING(getLogger("TemporaryFileOnLocalDisk"), "Temporary path '{}' does not exist in '{}' on disk {}", path_to_file, disk->getPath(), disk->getName());
}
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
private:
DiskPtr disk;
String path_to_file;
};
TemporaryFileProvider createTemporaryFileProvider(VolumePtr volume)
{
if (!volume)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Volume is not initialized");
return [volume](size_t max_size) -> std::unique_ptr<TemporaryFileHolder>
{
return std::make_unique<TemporaryFileOnLocalDisk>(volume, max_size);
};
}
TemporaryFileProvider createTemporaryFileProvider(FileCache * file_cache)
{
if (!file_cache || !file_cache->isInitialized())
throw Exception(ErrorCodes::LOGICAL_ERROR, "File cache is not initialized");
return [file_cache](size_t max_size) -> std::unique_ptr<TemporaryFileHolder>
{
return std::make_unique<TemporaryFileInLocalCache>(*file_cache, max_size);
};
}
TemporaryDataOnDiskScopePtr TemporaryDataOnDiskScope::childScope(CurrentMetrics::Metric current_metric)
{
TemporaryDataOnDiskSettings child_settings = settings;
child_settings.current_metric = current_metric;
return std::make_shared<TemporaryDataOnDiskScope>(shared_from_this(), child_settings);
}
TemporaryDataReadBuffer::TemporaryDataReadBuffer(std::unique_ptr<ReadBuffer> in_)
: ReadBuffer(nullptr, 0)
, compressed_buf(std::move(in_))
{
BufferBase::set(compressed_buf->buffer().begin(), compressed_buf->buffer().size(), compressed_buf->offset());
}
bool TemporaryDataReadBuffer::nextImpl()
{
compressed_buf->position() = position();
if (!compressed_buf->next())
{
set(compressed_buf->position(), 0);
return false;
}
BufferBase::set(compressed_buf->buffer().begin(), compressed_buf->buffer().size(), compressed_buf->offset());
return true;
}
TemporaryDataBuffer::TemporaryDataBuffer(TemporaryDataOnDiskScope * parent_, size_t reserve_size)
: WriteBuffer(nullptr, 0)
, parent(parent_)
, file_holder(parent->file_provider(reserve_size))
, out_compressed_buf(file_holder->write(), getCodec(parent->getSettings()))
{
WriteBuffer::set(out_compressed_buf->buffer().begin(), out_compressed_buf->buffer().size());
}
void TemporaryDataBuffer::nextImpl()
{
if (!out_compressed_buf)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file buffer writing has been finished");
out_compressed_buf->position() = position();
out_compressed_buf->next();
BufferBase::set(out_compressed_buf->buffer().begin(), out_compressed_buf->buffer().size(), out_compressed_buf->offset());
updateAllocAndCheck();
}
String TemporaryDataBuffer::describeFilePath() const
{
return file_holder->describeFilePath();
}
TemporaryDataBuffer::~TemporaryDataBuffer()
{
if (out_compressed_buf)
// read() nor finishWriting() was called
cancel();
}
void TemporaryDataBuffer::cancelImpl() noexcept
{
if (out_compressed_buf)
{
/// CompressedWriteBuffer doesn't call cancel/finalize for wrapped buffer
out_compressed_buf->cancel();
out_compressed_buf.getHolder()->cancel();
out_compressed_buf.reset();
}
}
void TemporaryDataBuffer::finalizeImpl()
{
if (!out_compressed_buf)
return;
/// CompressedWriteBuffer doesn't call cancel/finalize for wrapped buffer
out_compressed_buf->finalize();
out_compressed_buf.getHolder()->finalize();
updateAllocAndCheck();
out_compressed_buf.reset();
}
TemporaryDataBuffer::Stat TemporaryDataBuffer::finishWriting()
{
/// TemporaryDataBuffer::read can be called from multiple threads
std::call_once(write_finished, [this]
{
if (canceled)
throw Exception(ErrorCodes::INVALID_STATE, "Writing to temporary file buffer was not successful");
next();
finalize();
});
return stat;
}
std::unique_ptr<ReadBuffer> TemporaryDataBuffer::read()
{
finishWriting();
if (stat.compressed_size == 0 && stat.uncompressed_size == 0)
return std::make_unique<TemporaryDataReadBuffer>(std::make_unique<ReadBufferFromEmptyFile>());
/// Keep buffer size less that file size, to avoid memory overhead for large amounts of small files
size_t buffer_size = std::min<size_t>(stat.compressed_size, DBMS_DEFAULT_BUFFER_SIZE);
return std::make_unique<TemporaryDataReadBuffer>(file_holder->read(buffer_size));
}
void TemporaryDataBuffer::updateAllocAndCheck()
{
if (!out_compressed_buf)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file buffer writing has been finished");
size_t new_compressed_size = out_compressed_buf->getCompressedBytes();
size_t new_uncompressed_size = out_compressed_buf->getUncompressedBytes();
if (unlikely(new_compressed_size < stat.compressed_size || new_uncompressed_size < stat.uncompressed_size))
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Temporary file {} size decreased after write: compressed: {} -> {}, uncompressed: {} -> {}",
file_holder ? file_holder->describeFilePath() : "NULL",
new_compressed_size, stat.compressed_size, new_uncompressed_size, stat.uncompressed_size);
}
parent->deltaAllocAndCheck(new_compressed_size - stat.compressed_size, new_uncompressed_size - stat.uncompressed_size);
stat.compressed_size = new_compressed_size;
stat.uncompressed_size = new_uncompressed_size;
}
void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta) void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta)
{ {
@ -54,391 +339,25 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz
stat.uncompressed_size += uncompressed_delta; stat.uncompressed_size += uncompressed_delta;
} }
TemporaryDataOnDisk::TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_) TemporaryBlockStreamHolder::TemporaryBlockStreamHolder(const Block & header_, TemporaryDataOnDiskScope * parent_, size_t reserve_size)
: TemporaryDataOnDiskScope(parent_, parent_->getSettings()) : WrapperGuard(std::make_unique<TemporaryDataBuffer>(parent_, reserve_size), DBMS_TCP_PROTOCOL_VERSION, header_)
, header(header_)
{} {}
TemporaryDataOnDisk::TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_, CurrentMetrics::Metric metric_scope) TemporaryDataBuffer::Stat TemporaryBlockStreamHolder::finishWriting() const
: TemporaryDataOnDiskScope(parent_, parent_->getSettings())
, current_metric_scope(metric_scope)
{}
std::unique_ptr<WriteBufferFromFileBase> TemporaryDataOnDisk::createRawStream(size_t max_file_size)
{ {
if (file_cache && file_cache->isInitialized()) if (!holder)
{ throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary block stream is not initialized");
auto holder = createCacheFile(max_file_size);
return std::make_unique<WriteBufferToFileSegment>(std::move(holder));
}
if (volume)
{
auto tmp_file = createRegularFile(max_file_size);
return std::make_unique<WriteBufferFromTemporaryFile>(std::move(tmp_file));
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryDataOnDiskScope has no cache and no volume"); impl->flush();
return holder->finishWriting();
} }
TemporaryFileStream & TemporaryDataOnDisk::createStream(const Block & header, size_t max_file_size) TemporaryBlockStreamReaderHolder TemporaryBlockStreamHolder::getReadStream() const
{ {
if (file_cache && file_cache->isInitialized()) if (!holder)
{ throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary block stream is not initialized");
auto holder = createCacheFile(max_file_size); return TemporaryBlockStreamReaderHolder(holder->read(), header, DBMS_TCP_PROTOCOL_VERSION);
std::lock_guard lock(mutex);
TemporaryFileStreamPtr & tmp_stream = streams.emplace_back(std::make_unique<TemporaryFileStream>(std::move(holder), header, this));
return *tmp_stream;
}
if (volume)
{
auto tmp_file = createRegularFile(max_file_size);
std::lock_guard lock(mutex);
TemporaryFileStreamPtr & tmp_stream
= streams.emplace_back(std::make_unique<TemporaryFileStream>(std::move(tmp_file), header, this));
return *tmp_stream;
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryDataOnDiskScope has no cache and no volume");
}
FileSegmentsHolderPtr TemporaryDataOnDisk::createCacheFile(size_t max_file_size)
{
if (!file_cache)
throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryDataOnDiskScope has no cache");
ProfileEvents::increment(ProfileEvents::ExternalProcessingFilesTotal);
const auto key = FileSegment::Key::random();
auto holder = file_cache->set(
key, 0, std::max(10_MiB, max_file_size),
CreateFileSegmentSettings(FileSegmentKind::Ephemeral), FileCache::getCommonUser());
chassert(holder->size() == 1);
holder->back().getKeyMetadata()->createBaseDirectory(/* throw_if_failed */true);
return holder;
}
TemporaryFileOnDiskHolder TemporaryDataOnDisk::createRegularFile(size_t max_file_size)
{
if (!volume)
throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryDataOnDiskScope has no volume");
DiskPtr disk;
if (max_file_size > 0)
{
auto reservation = volume->reserve(max_file_size);
if (!reservation)
throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Not enough space on temporary disk");
disk = reservation->getDisk();
}
else
{
disk = volume->getDisk();
}
/// We do not increment ProfileEvents::ExternalProcessingFilesTotal here because it is incremented in TemporaryFileOnDisk constructor.
return std::make_unique<TemporaryFileOnDisk>(disk, current_metric_scope);
}
std::vector<TemporaryFileStream *> TemporaryDataOnDisk::getStreams() const
{
std::vector<TemporaryFileStream *> res;
std::lock_guard lock(mutex);
res.reserve(streams.size());
for (const auto & stream : streams)
res.push_back(stream.get());
return res;
}
bool TemporaryDataOnDisk::empty() const
{
std::lock_guard lock(mutex);
return streams.empty();
}
static inline CompressionCodecPtr getCodec(const TemporaryDataOnDiskSettings & settings)
{
if (settings.compression_codec.empty())
return CompressionCodecFactory::instance().get("NONE");
return CompressionCodecFactory::instance().get(settings.compression_codec);
}
struct TemporaryFileStream::OutputWriter
{
OutputWriter(std::unique_ptr<WriteBuffer> out_buf_, const Block & header_, const TemporaryDataOnDiskSettings & settings)
: out_buf(std::move(out_buf_))
, out_compressed_buf(*out_buf, getCodec(settings))
, out_writer(out_compressed_buf, DBMS_TCP_PROTOCOL_VERSION, header_)
{
}
size_t write(const Block & block)
{
if (finalized)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write to finalized stream");
size_t written_bytes = out_writer.write(block);
num_rows += block.rows();
return written_bytes;
}
void flush()
{
if (finalized)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot flush finalized stream");
out_compressed_buf.next();
out_buf->next();
out_writer.flush();
}
void finalize()
{
if (finalized)
return;
/// if we called finalize() explicitly, and got an exception,
/// we don't want to get it again in the destructor, so set finalized flag first
finalized = true;
out_writer.flush();
out_compressed_buf.finalize();
out_buf->finalize();
}
~OutputWriter()
{
try
{
finalize();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
std::unique_ptr<WriteBuffer> out_buf;
CompressedWriteBuffer out_compressed_buf;
NativeWriter out_writer;
std::atomic_size_t num_rows = 0;
bool finalized = false;
};
TemporaryFileStream::Reader::Reader(const String & path_, const Block & header_, size_t size_)
: path(path_)
, size(size_ ? std::min<size_t>(size_, DBMS_DEFAULT_BUFFER_SIZE) : DBMS_DEFAULT_BUFFER_SIZE)
, header(header_)
{
LOG_TEST(getLogger("TemporaryFileStream"), "Reading {} from {}", header_.dumpStructure(), path);
}
TemporaryFileStream::Reader::Reader(const String & path_, size_t size_)
: path(path_)
, size(size_ ? std::min<size_t>(size_, DBMS_DEFAULT_BUFFER_SIZE) : DBMS_DEFAULT_BUFFER_SIZE)
{
LOG_TEST(getLogger("TemporaryFileStream"), "Reading from {}", path);
}
Block TemporaryFileStream::Reader::read()
{
if (!in_reader)
{
if (fs::exists(path))
in_file_buf = std::make_unique<ReadBufferFromFile>(path, size);
else
in_file_buf = std::make_unique<ReadBufferFromEmptyFile>();
in_compressed_buf = std::make_unique<CompressedReadBuffer>(*in_file_buf);
if (header.has_value())
in_reader = std::make_unique<NativeReader>(*in_compressed_buf, header.value(), DBMS_TCP_PROTOCOL_VERSION);
else
in_reader = std::make_unique<NativeReader>(*in_compressed_buf, DBMS_TCP_PROTOCOL_VERSION);
}
return in_reader->read();
}
TemporaryFileStream::TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_)
: parent(parent_)
, header(header_)
, file(std::move(file_))
, out_writer(std::make_unique<OutputWriter>(std::make_unique<WriteBufferFromFile>(file->getAbsolutePath()), header, parent->settings))
{
LOG_TEST(getLogger("TemporaryFileStream"), "Writing to temporary file {}", file->getAbsolutePath());
}
TemporaryFileStream::TemporaryFileStream(FileSegmentsHolderPtr segments_, const Block & header_, TemporaryDataOnDisk * parent_)
: parent(parent_)
, header(header_)
, segment_holder(std::move(segments_))
{
if (segment_holder->size() != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryFileStream can be created only from single segment");
auto out_buf = std::make_unique<WriteBufferToFileSegment>(&segment_holder->front());
LOG_TEST(getLogger("TemporaryFileStream"), "Writing to temporary file {}", out_buf->getFileName());
out_writer = std::make_unique<OutputWriter>(std::move(out_buf), header, parent_->settings);
}
size_t TemporaryFileStream::write(const Block & block)
{
if (!out_writer)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing has been finished");
updateAllocAndCheck();
size_t bytes_written = out_writer->write(block);
return bytes_written;
}
void TemporaryFileStream::flush()
{
if (!out_writer)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing has been finished");
out_writer->flush();
}
TemporaryFileStream::Stat TemporaryFileStream::finishWriting()
{
if (isWriteFinished())
return stat;
if (out_writer)
{
out_writer->finalize();
/// The amount of written data can be changed after finalization, some buffers can be flushed
/// Need to update the stat
updateAllocAndCheck();
out_writer.reset();
/// reader will be created at the first read call, not to consume memory before it is needed
}
return stat;
}
TemporaryFileStream::Stat TemporaryFileStream::finishWritingAsyncSafe()
{
std::call_once(finish_writing, [this]{ finishWriting(); });
return stat;
}
bool TemporaryFileStream::isWriteFinished() const
{
assert(in_reader == nullptr || out_writer == nullptr);
return out_writer == nullptr;
}
Block TemporaryFileStream::read()
{
if (!isWriteFinished())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing has been not finished");
if (isEof())
return {};
if (!in_reader)
{
in_reader = std::make_unique<Reader>(getPath(), header, getSize());
}
Block block = in_reader->read();
if (!block)
{
/// finalize earlier to release resources, do not wait for the destructor
this->release();
}
return block;
}
std::unique_ptr<TemporaryFileStream::Reader> TemporaryFileStream::getReadStream()
{
if (!isWriteFinished())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Writing has been not finished");
if (isEof())
return nullptr;
return std::make_unique<Reader>(getPath(), header, getSize());
}
void TemporaryFileStream::updateAllocAndCheck()
{
assert(out_writer);
size_t new_compressed_size = out_writer->out_compressed_buf.getCompressedBytes();
size_t new_uncompressed_size = out_writer->out_compressed_buf.getUncompressedBytes();
if (unlikely(new_compressed_size < stat.compressed_size || new_uncompressed_size < stat.uncompressed_size))
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Temporary file {} size decreased after write: compressed: {} -> {}, uncompressed: {} -> {}",
getPath(), new_compressed_size, stat.compressed_size, new_uncompressed_size, stat.uncompressed_size);
}
parent->deltaAllocAndCheck(new_compressed_size - stat.compressed_size, new_uncompressed_size - stat.uncompressed_size);
stat.compressed_size = new_compressed_size;
stat.uncompressed_size = new_uncompressed_size;
stat.num_rows = out_writer->num_rows;
}
bool TemporaryFileStream::isEof() const
{
return file == nullptr && !segment_holder;
}
void TemporaryFileStream::release()
{
if (in_reader)
in_reader.reset();
if (out_writer)
{
out_writer->finalize();
out_writer.reset();
}
if (file)
{
file.reset();
parent->deltaAllocAndCheck(-stat.compressed_size, -stat.uncompressed_size);
}
if (segment_holder)
segment_holder.reset();
}
String TemporaryFileStream::getPath() const
{
if (file)
return file->getAbsolutePath();
if (segment_holder && !segment_holder->empty())
return segment_holder->front().getPath();
throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryFileStream has no file");
}
size_t TemporaryFileStream::getSize() const
{
if (file)
return file->getDisk()->getFileSize(file->getRelativePath());
if (segment_holder && !segment_holder->empty())
return segment_holder->front().getReservedSize();
throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryFileStream has no file");
}
TemporaryFileStream::~TemporaryFileStream()
{
try
{
release();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
assert(false); /// deltaAllocAndCheck with negative can't throw exception
}
} }
} }

View File

@ -4,15 +4,21 @@
#include <mutex> #include <mutex>
#include <boost/noncopyable.hpp> #include <boost/noncopyable.hpp>
#include <IO/ReadBufferFromFile.h> #include <Common/CurrentMetrics.h>
#include <Compression/CompressedReadBuffer.h> #include <Compression/CompressedReadBuffer.h>
#include <Formats/NativeReader.h> #include <Compression/CompressedWriteBuffer.h>
#include <Core/Block.h>
#include <Disks/IVolume.h> #include <Disks/IVolume.h>
#include <Disks/TemporaryFileOnDisk.h> #include <Disks/TemporaryFileOnDisk.h>
#include <Interpreters/Cache/FileSegment.h>
#include <Common/CurrentMetrics.h>
#include <Formats/NativeReader.h>
#include <Formats/NativeWriter.h>
#include <Interpreters/Cache/FileSegment.h>
#include <IO/ReadBufferFromFile.h>
class FileCacheTest_TemporaryDataReadBufferSize_Test;
namespace CurrentMetrics namespace CurrentMetrics
{ {
@ -25,11 +31,10 @@ namespace DB
class TemporaryDataOnDiskScope; class TemporaryDataOnDiskScope;
using TemporaryDataOnDiskScopePtr = std::shared_ptr<TemporaryDataOnDiskScope>; using TemporaryDataOnDiskScopePtr = std::shared_ptr<TemporaryDataOnDiskScope>;
class TemporaryDataOnDisk; class TemporaryDataBuffer;
using TemporaryDataOnDiskPtr = std::unique_ptr<TemporaryDataOnDisk>; using TemporaryDataBufferPtr = std::unique_ptr<TemporaryDataBuffer>;
class TemporaryFileStream; class TemporaryFileHolder;
using TemporaryFileStreamPtr = std::unique_ptr<TemporaryFileStream>;
class FileCache; class FileCache;
@ -40,15 +45,26 @@ struct TemporaryDataOnDiskSettings
/// Compression codec for temporary data, if empty no compression will be used. LZ4 by default /// Compression codec for temporary data, if empty no compression will be used. LZ4 by default
String compression_codec = "LZ4"; String compression_codec = "LZ4";
/// Read/Write internal buffer size
size_t buffer_size = DBMS_DEFAULT_BUFFER_SIZE;
/// Metrics counter to increment when temporary file in current scope are created
CurrentMetrics::Metric current_metric = CurrentMetrics::TemporaryFilesUnknown;
}; };
/// Creates temporary files located on specified resource (disk, fs_cache, etc.)
using TemporaryFileProvider = std::function<std::unique_ptr<TemporaryFileHolder>(size_t)>;
TemporaryFileProvider createTemporaryFileProvider(VolumePtr volume);
TemporaryFileProvider createTemporaryFileProvider(FileCache * file_cache);
/* /*
* Used to account amount of temporary data written to disk. * Used to account amount of temporary data written to disk.
* If limit is set, throws exception if limit is exceeded. * If limit is set, throws exception if limit is exceeded.
* Data can be nested, so parent scope accounts all data written by children. * Data can be nested, so parent scope accounts all data written by children.
* Scopes are: global -> per-user -> per-query -> per-purpose (sorting, aggregation, etc). * Scopes are: global -> per-user -> per-query -> per-purpose (sorting, aggregation, etc).
*/ */
class TemporaryDataOnDiskScope : boost::noncopyable class TemporaryDataOnDiskScope : boost::noncopyable, public std::enable_shared_from_this<TemporaryDataOnDiskScope>
{ {
public: public:
struct StatAtomic struct StatAtomic
@ -57,164 +73,156 @@ public:
std::atomic<size_t> uncompressed_size; std::atomic<size_t> uncompressed_size;
}; };
explicit TemporaryDataOnDiskScope(VolumePtr volume_, TemporaryDataOnDiskSettings settings_) /// Root scope
: volume(std::move(volume_)) template <typename T>
TemporaryDataOnDiskScope(T && storage, TemporaryDataOnDiskSettings settings_)
: file_provider(createTemporaryFileProvider(std::forward<T>(storage)))
, settings(std::move(settings_)) , settings(std::move(settings_))
{} {}
explicit TemporaryDataOnDiskScope(VolumePtr volume_, FileCache * file_cache_, TemporaryDataOnDiskSettings settings_)
: volume(std::move(volume_))
, file_cache(file_cache_)
, settings(std::move(settings_))
{}
explicit TemporaryDataOnDiskScope(TemporaryDataOnDiskScopePtr parent_, TemporaryDataOnDiskSettings settings_) TemporaryDataOnDiskScope(TemporaryDataOnDiskScopePtr parent_, TemporaryDataOnDiskSettings settings_)
: parent(std::move(parent_)) : parent(std::move(parent_))
, volume(parent->volume) , file_provider(parent->file_provider)
, file_cache(parent->file_cache)
, settings(std::move(settings_)) , settings(std::move(settings_))
{} {}
/// TODO: remove TemporaryDataOnDiskScopePtr childScope(CurrentMetrics::Metric current_metric);
/// Refactor all code that uses volume directly to use TemporaryDataOnDisk.
VolumePtr getVolume() const { return volume; }
const TemporaryDataOnDiskSettings & getSettings() const { return settings; } const TemporaryDataOnDiskSettings & getSettings() const { return settings; }
protected: protected:
friend class TemporaryDataBuffer;
void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta); void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta);
TemporaryDataOnDiskScopePtr parent = nullptr; TemporaryDataOnDiskScopePtr parent = nullptr;
VolumePtr volume = nullptr; TemporaryFileProvider file_provider;
FileCache * file_cache = nullptr;
StatAtomic stat; StatAtomic stat;
const TemporaryDataOnDiskSettings settings; const TemporaryDataOnDiskSettings settings;
}; };
/* /** Used to hold the wrapper and wrapped object together.
* Holds the set of temporary files. * This class provides a convenient way to manage the lifetime of both the wrapper and the wrapped object.
* New file stream is created with `createStream`. * The wrapper class (Impl) stores a reference to the wrapped object (Holder), and both objects are owned by this class.
* Streams are owned by this object and will be deleted when it is deleted. * The lifetime of the wrapper and the wrapped object should be the same.
* It's a leaf node in temporary data scope tree. * This pattern is commonly used when the caller only needs to interact with the wrapper and doesn't need to be aware of the wrapped object.
*/ * Examples: CompressedWriteBuffer and WriteBuffer, and NativeReader and ReadBuffer.
class TemporaryDataOnDisk : private TemporaryDataOnDiskScope */
template <typename Impl, typename Holder>
class WrapperGuard
{ {
friend class TemporaryFileStream; /// to allow it to call `deltaAllocAndCheck` to account data
public: public:
using TemporaryDataOnDiskScope::StatAtomic; template <typename ... Args>
explicit WrapperGuard(std::unique_ptr<Holder> holder_, Args && ... args)
: holder(std::move(holder_))
, impl(std::make_unique<Impl>(*holder, std::forward<Args>(args)...))
{
chassert(holder);
chassert(impl);
}
explicit TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_); Impl * operator->() { chassert(impl); chassert(holder); return impl.get(); }
const Impl * operator->() const { chassert(impl); chassert(holder); return impl.get(); }
Impl & operator*() { chassert(impl); chassert(holder); return *impl; }
const Impl & operator*() const { chassert(impl); chassert(holder); return *impl; }
operator bool() const { return impl != nullptr; } /// NOLINT
explicit TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_, CurrentMetrics::Metric metric_scope); const Holder * getHolder() const { return holder.get(); }
Holder * getHolder() { return holder.get(); }
/// If max_file_size > 0, then check that there's enough space on the disk and throw an exception in case of lack of free space void reset()
TemporaryFileStream & createStream(const Block & header, size_t max_file_size = 0); {
impl.reset();
holder.reset();
}
/// Write raw data directly into buffer. protected:
/// Differences from `createStream`: std::unique_ptr<Holder> holder;
/// 1) it doesn't account data in parent scope std::unique_ptr<Impl> impl;
/// 2) returned buffer owns resources (instead of TemporaryDataOnDisk itself)
/// If max_file_size > 0, then check that there's enough space on the disk and throw an exception in case of lack of free space
std::unique_ptr<WriteBufferFromFileBase> createRawStream(size_t max_file_size = 0);
std::vector<TemporaryFileStream *> getStreams() const;
bool empty() const;
const StatAtomic & getStat() const { return stat; }
private:
FileSegmentsHolderPtr createCacheFile(size_t max_file_size);
TemporaryFileOnDiskHolder createRegularFile(size_t max_file_size);
mutable std::mutex mutex;
std::vector<TemporaryFileStreamPtr> streams TSA_GUARDED_BY(mutex);
typename CurrentMetrics::Metric current_metric_scope = CurrentMetrics::TemporaryFilesUnknown;
}; };
/* /// Owns temporary file and provides access to it.
* Data can be written into this stream and then read. /// On destruction, file is removed and all resources are freed.
* After finish writing, call `finishWriting` and then either call `read` or 'getReadStream'(only one of the two) to read the data. /// Lifetime of read/write buffers should be less than lifetime of TemporaryFileHolder.
* Account amount of data written to disk in parent scope. class TemporaryFileHolder
*/
class TemporaryFileStream : boost::noncopyable
{ {
public: public:
struct Reader TemporaryFileHolder();
{
Reader(const String & path, const Block & header_, size_t size = 0);
explicit Reader(const String & path, size_t size = 0); virtual std::unique_ptr<WriteBuffer> write() = 0;
virtual std::unique_ptr<ReadBuffer> read(size_t buffer_size) const = 0;
Block read(); /// Get location for logging
virtual String describeFilePath() const = 0;
const std::string path; virtual ~TemporaryFileHolder() = default;
const size_t size; };
const std::optional<Block> header;
std::unique_ptr<ReadBufferFromFileBase> in_file_buf; /// Reads raw data from temporary file
std::unique_ptr<CompressedReadBuffer> in_compressed_buf; class TemporaryDataReadBuffer : public ReadBuffer
std::unique_ptr<NativeReader> in_reader; {
}; public:
explicit TemporaryDataReadBuffer(std::unique_ptr<ReadBuffer> in_);
private:
friend class ::FileCacheTest_TemporaryDataReadBufferSize_Test;
bool nextImpl() override;
WrapperGuard<CompressedReadBuffer, ReadBuffer> compressed_buf;
};
/// Writes raw data to buffer provided by file_holder, and accounts amount of written data in parent scope.
class TemporaryDataBuffer : public WriteBuffer
{
public:
struct Stat struct Stat
{ {
/// Statistics for file
/// Non-atomic because we don't allow to `read` or `write` into single file from multiple threads
size_t compressed_size = 0; size_t compressed_size = 0;
size_t uncompressed_size = 0; size_t uncompressed_size = 0;
size_t num_rows = 0;
}; };
TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_); explicit TemporaryDataBuffer(TemporaryDataOnDiskScope * parent_, size_t reserve_size = 0);
TemporaryFileStream(FileSegmentsHolderPtr segments_, const Block & header_, TemporaryDataOnDisk * parent_); void nextImpl() override;
void finalizeImpl() override;
size_t write(const Block & block); void cancelImpl() noexcept override;
void flush();
std::unique_ptr<ReadBuffer> read();
Stat finishWriting(); Stat finishWriting();
Stat finishWritingAsyncSafe();
bool isWriteFinished() const;
std::unique_ptr<Reader> getReadStream(); String describeFilePath() const;
Block read(); ~TemporaryDataBuffer() override;
String getPath() const;
size_t getSize() const;
Block getHeader() const { return header; }
/// Read finished and file released
bool isEof() const;
~TemporaryFileStream();
private: private:
void updateAllocAndCheck(); void updateAllocAndCheck();
/// Release everything, close reader and writer, delete file TemporaryDataOnDiskScope * parent;
void release(); std::unique_ptr<TemporaryFileHolder> file_holder;
WrapperGuard<CompressedWriteBuffer, WriteBuffer> out_compressed_buf;
TemporaryDataOnDisk * parent; std::once_flag write_finished;
Block header;
/// Data can be stored in file directly or in the cache
TemporaryFileOnDiskHolder file;
FileSegmentsHolderPtr segment_holder;
Stat stat; Stat stat;
};
std::once_flag finish_writing;
struct OutputWriter; /// High level interfaces for reading and writing temporary data by blocks.
std::unique_ptr<OutputWriter> out_writer; using TemporaryBlockStreamReaderHolder = WrapperGuard<NativeReader, ReadBuffer>;
std::unique_ptr<Reader> in_reader; class TemporaryBlockStreamHolder : public WrapperGuard<NativeWriter, TemporaryDataBuffer>
{
public:
TemporaryBlockStreamHolder(const Block & header_, TemporaryDataOnDiskScope * parent_, size_t reserve_size = 0);
TemporaryBlockStreamReaderHolder getReadStream() const;
TemporaryDataBuffer::Stat finishWriting() const;
const Block & getHeader() const { return header; }
private:
Block header;
}; };
} }

View File

@ -934,7 +934,7 @@ static Block generateBlock(size_t size = 0)
return block; return block;
} }
static size_t readAllTemporaryData(TemporaryFileStream & stream) static size_t readAllTemporaryData(NativeReader & stream)
{ {
Block block; Block block;
size_t read_rows = 0; size_t read_rows = 0;
@ -947,6 +947,7 @@ static size_t readAllTemporaryData(TemporaryFileStream & stream)
} }
TEST_F(FileCacheTest, temporaryData) TEST_F(FileCacheTest, temporaryData)
try
{ {
ServerUUID::setRandomForUnitTests(); ServerUUID::setRandomForUnitTests();
DB::FileCacheSettings settings; DB::FileCacheSettings settings;
@ -959,7 +960,7 @@ TEST_F(FileCacheTest, temporaryData)
file_cache.initialize(); file_cache.initialize();
const auto user = FileCache::getCommonUser(); const auto user = FileCache::getCommonUser();
auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(nullptr, &file_cache, TemporaryDataOnDiskSettings{}); auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(&file_cache, TemporaryDataOnDiskSettings{});
auto some_data_holder = file_cache.getOrSet(FileCacheKey::fromPath("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}, 0, user); auto some_data_holder = file_cache.getOrSet(FileCacheKey::fromPath("some_data"), 0, 5_KiB, 5_KiB, CreateFileSegmentSettings{}, 0, user);
@ -982,12 +983,17 @@ TEST_F(FileCacheTest, temporaryData)
size_t size_used_with_temporary_data; size_t size_used_with_temporary_data;
size_t segments_used_with_temporary_data; size_t segments_used_with_temporary_data;
{ {
auto tmp_data = std::make_unique<TemporaryDataOnDisk>(tmp_data_scope); TemporaryBlockStreamHolder stream(generateBlock(), tmp_data_scope.get());
ASSERT_TRUE(stream);
/// Do nothing with stream, just create it and destroy.
}
auto & stream = tmp_data->createStream(generateBlock()); {
TemporaryBlockStreamHolder stream(generateBlock(), tmp_data_scope.get());
ASSERT_GT(stream.write(generateBlock(100)), 0); ASSERT_GT(stream->write(generateBlock(100)), 0);
ASSERT_GT(file_cache.getUsedCacheSize(), 0); ASSERT_GT(file_cache.getUsedCacheSize(), 0);
ASSERT_GT(file_cache.getFileSegmentsNum(), 0); ASSERT_GT(file_cache.getFileSegmentsNum(), 0);
@ -995,22 +1001,22 @@ TEST_F(FileCacheTest, temporaryData)
size_t used_size_before_attempt = file_cache.getUsedCacheSize(); size_t used_size_before_attempt = file_cache.getUsedCacheSize();
/// data can't be evicted because it is still held by `some_data_holder` /// data can't be evicted because it is still held by `some_data_holder`
ASSERT_THROW({ ASSERT_THROW({
stream.write(generateBlock(2000)); stream->write(generateBlock(2000));
stream.flush(); stream.finishWriting();
}, DB::Exception); }, DB::Exception);
ASSERT_THROW(stream.finishWriting(), DB::Exception);
ASSERT_EQ(file_cache.getUsedCacheSize(), used_size_before_attempt); ASSERT_EQ(file_cache.getUsedCacheSize(), used_size_before_attempt);
} }
{ {
size_t before_used_size = file_cache.getUsedCacheSize(); size_t before_used_size = file_cache.getUsedCacheSize();
auto tmp_data = std::make_unique<TemporaryDataOnDisk>(tmp_data_scope); auto write_buf_stream = std::make_unique<TemporaryDataBuffer>(tmp_data_scope.get());
auto write_buf_stream = tmp_data->createRawStream();
write_buf_stream->write("1234567890", 10); write_buf_stream->write("1234567890", 10);
write_buf_stream->write("abcde", 5); write_buf_stream->write("abcde", 5);
auto read_buf = dynamic_cast<IReadableWriteBuffer *>(write_buf_stream.get())->tryGetReadBuffer(); auto read_buf = write_buf_stream->read();
ASSERT_GT(file_cache.getUsedCacheSize(), before_used_size + 10); ASSERT_GT(file_cache.getUsedCacheSize(), before_used_size + 10);
@ -1023,22 +1029,22 @@ TEST_F(FileCacheTest, temporaryData)
} }
{ {
auto tmp_data = std::make_unique<TemporaryDataOnDisk>(tmp_data_scope); TemporaryBlockStreamHolder stream(generateBlock(), tmp_data_scope.get());
auto & stream = tmp_data->createStream(generateBlock());
ASSERT_GT(stream.write(generateBlock(100)), 0); ASSERT_GT(stream->write(generateBlock(100)), 0);
some_data_holder.reset(); some_data_holder.reset();
stream.write(generateBlock(2000)); stream->write(generateBlock(2000));
auto stat = stream.finishWriting(); stream.finishWriting();
ASSERT_TRUE(fs::exists(stream.getPath())); String file_path = stream.getHolder()->describeFilePath().substr(strlen("fscache://"));
ASSERT_GT(fs::file_size(stream.getPath()), 100);
ASSERT_EQ(stat.num_rows, 2100); ASSERT_TRUE(fs::exists(file_path)) << "File " << file_path << " should exist";
ASSERT_EQ(readAllTemporaryData(stream), 2100); ASSERT_GT(fs::file_size(file_path), 100) << "File " << file_path << " should be larger than 100 bytes";
ASSERT_EQ(readAllTemporaryData(*stream.getReadStream()), 2100);
size_used_with_temporary_data = file_cache.getUsedCacheSize(); size_used_with_temporary_data = file_cache.getUsedCacheSize();
segments_used_with_temporary_data = file_cache.getFileSegmentsNum(); segments_used_with_temporary_data = file_cache.getFileSegmentsNum();
@ -1054,6 +1060,11 @@ TEST_F(FileCacheTest, temporaryData)
ASSERT_LE(file_cache.getUsedCacheSize(), size_used_before_temporary_data); ASSERT_LE(file_cache.getUsedCacheSize(), size_used_before_temporary_data);
ASSERT_LE(file_cache.getFileSegmentsNum(), segments_used_before_temporary_data); ASSERT_LE(file_cache.getFileSegmentsNum(), segments_used_before_temporary_data);
} }
catch (...)
{
std::cerr << getCurrentExceptionMessage(true) << std::endl;
throw;
}
TEST_F(FileCacheTest, CachedReadBuffer) TEST_F(FileCacheTest, CachedReadBuffer)
{ {
@ -1148,18 +1159,22 @@ TEST_F(FileCacheTest, TemporaryDataReadBufferSize)
DB::FileCache file_cache("cache", settings); DB::FileCache file_cache("cache", settings);
file_cache.initialize(); file_cache.initialize();
auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(/*volume=*/nullptr, &file_cache, /*settings=*/TemporaryDataOnDiskSettings{}); auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(&file_cache, TemporaryDataOnDiskSettings{});
auto tmp_data = std::make_unique<TemporaryDataOnDisk>(tmp_data_scope);
auto block = generateBlock(/*size=*/3); auto block = generateBlock(/*size=*/3);
auto & stream = tmp_data->createStream(block); TemporaryBlockStreamHolder stream(block, tmp_data_scope.get());
stream.write(block);
stream.finishWriting();
/// We allocate buffer of size min(getSize(), DBMS_DEFAULT_BUFFER_SIZE) stream->write(block);
auto stat = stream.finishWriting();
/// We allocate buffer of size min(stat.compressed_size, DBMS_DEFAULT_BUFFER_SIZE)
/// We do care about buffer size because realistic external group by could generate 10^5 temporary files /// We do care about buffer size because realistic external group by could generate 10^5 temporary files
ASSERT_EQ(stream.getSize(), 62); ASSERT_EQ(stat.compressed_size, 62);
auto reader = stream.getReadStream();
auto * read_buf = reader.getHolder();
const auto & internal_buffer = static_cast<TemporaryDataReadBuffer *>(read_buf)->compressed_buf.getHolder()->internalBuffer();
ASSERT_EQ(internal_buffer.size(), 62);
} }
/// Temporary data stored on disk /// Temporary data stored on disk
@ -1170,16 +1185,14 @@ TEST_F(FileCacheTest, TemporaryDataReadBufferSize)
disk = createDisk("temporary_data_read_buffer_size_test_dir"); disk = createDisk("temporary_data_read_buffer_size_test_dir");
VolumePtr volume = std::make_shared<SingleDiskVolume>("volume", disk); VolumePtr volume = std::make_shared<SingleDiskVolume>("volume", disk);
auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(/*volume=*/volume, /*cache=*/nullptr, /*settings=*/TemporaryDataOnDiskSettings{}); auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(volume, TemporaryDataOnDiskSettings{});
auto tmp_data = std::make_unique<TemporaryDataOnDisk>(tmp_data_scope);
auto block = generateBlock(/*size=*/3); auto block = generateBlock(/*size=*/3);
auto & stream = tmp_data->createStream(block); TemporaryBlockStreamHolder stream(block, tmp_data_scope.get());
stream.write(block); stream->write(block);
stream.finishWriting(); auto stat = stream.finishWriting();
ASSERT_EQ(stream.getSize(), 62); ASSERT_EQ(stat.compressed_size, 62);
} }
} }

View File

@ -31,7 +31,7 @@ CreateQueryUUIDs::CreateQueryUUIDs(const ASTCreateQuery & query, bool generate_r
/// If we generate random UUIDs for already existing tables then those UUIDs will not be correct making those inner target table inaccessible. /// If we generate random UUIDs for already existing tables then those UUIDs will not be correct making those inner target table inaccessible.
/// Thus it's not safe for example to replace /// Thus it's not safe for example to replace
/// "ATTACH MATERIALIZED VIEW mv AS SELECT a FROM b" with /// "ATTACH MATERIALIZED VIEW mv AS SELECT a FROM b" with
/// "ATTACH MATERIALIZED VIEW mv TO INNER UUID "XXXX" AS SELECT a FROM b" /// "ATTACH MATERIALIZED VIEW mv TO INNER UUID "248372b7-02c4-4c88-a5e1-282a83cc572a" AS SELECT a FROM b"
/// This replacement is safe only for CREATE queries when inner target tables don't exist yet. /// This replacement is safe only for CREATE queries when inner target tables don't exist yet.
if (!query.attach) if (!query.attach)
{ {

View File

@ -274,7 +274,7 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr &
return res; return res;
} }
FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options) FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr & query_tree_node, const SelectQueryOptions & select_query_options)
{ {
if (select_query_options.only_analyze) if (select_query_options.only_analyze)
return {}; return {};

View File

@ -659,6 +659,7 @@ std::unique_ptr<ExpressionStep> createComputeAliasColumnsStep(
} }
JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression,
const QueryTreeNodePtr & parent_join_tree,
const SelectQueryInfo & select_query_info, const SelectQueryInfo & select_query_info,
const SelectQueryOptions & select_query_options, const SelectQueryOptions & select_query_options,
PlannerContextPtr & planner_context, PlannerContextPtr & planner_context,
@ -696,8 +697,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
table_expression_query_info.table_expression = table_expression; table_expression_query_info.table_expression = table_expression;
if (const auto & filter_actions = table_expression_data.getFilterActions()) if (const auto & filter_actions = table_expression_data.getFilterActions())
table_expression_query_info.filter_actions_dag = std::make_shared<const ActionsDAG>(filter_actions->clone()); table_expression_query_info.filter_actions_dag = std::make_shared<const ActionsDAG>(filter_actions->clone());
table_expression_query_info.current_table_chosen_for_reading_with_parallel_replicas
= table_node == planner_context->getGlobalPlannerContext()->parallel_replicas_table;
size_t max_streams = settings[Setting::max_threads]; size_t max_streams = settings[Setting::max_threads];
size_t max_threads_execute_query = settings[Setting::max_threads]; size_t max_threads_execute_query = settings[Setting::max_threads];
@ -912,21 +911,35 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
/// It is just a safety check needed until we have a proper sending plan to replicas. /// It is just a safety check needed until we have a proper sending plan to replicas.
/// If we have a non-trivial storage like View it might create its own Planner inside read(), run findTableForParallelReplicas() /// If we have a non-trivial storage like View it might create its own Planner inside read(), run findTableForParallelReplicas()
/// and find some other table that might be used for reading with parallel replicas. It will lead to errors. /// and find some other table that might be used for reading with parallel replicas. It will lead to errors.
const bool other_table_already_chosen_for_reading_with_parallel_replicas const bool no_tables_or_another_table_chosen_for_reading_with_parallel_replicas_mode
= planner_context->getGlobalPlannerContext()->parallel_replicas_table = query_context->canUseParallelReplicasOnFollower()
&& !table_expression_query_info.current_table_chosen_for_reading_with_parallel_replicas; && table_node != planner_context->getGlobalPlannerContext()->parallel_replicas_table;
if (other_table_already_chosen_for_reading_with_parallel_replicas) if (no_tables_or_another_table_chosen_for_reading_with_parallel_replicas_mode)
planner_context->getMutableQueryContext()->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); {
auto mutable_context = Context::createCopy(query_context);
storage->read( mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
query_plan, storage->read(
columns_names, query_plan,
storage_snapshot, columns_names,
table_expression_query_info, storage_snapshot,
query_context, table_expression_query_info,
from_stage, std::move(mutable_context),
max_block_size, from_stage,
max_streams); max_block_size,
max_streams);
}
else
{
storage->read(
query_plan,
columns_names,
storage_snapshot,
table_expression_query_info,
query_context,
from_stage,
max_block_size,
max_streams);
}
auto parallel_replicas_enabled_for_storage = [](const StoragePtr & table, const Settings & query_settings) auto parallel_replicas_enabled_for_storage = [](const StoragePtr & table, const Settings & query_settings)
{ {
@ -942,6 +955,19 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
/// query_plan can be empty if there is nothing to read /// query_plan can be empty if there is nothing to read
if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings)) if (query_plan.isInitialized() && parallel_replicas_enabled_for_storage(storage, settings))
{ {
const bool allow_parallel_replicas_for_table_expression = [](const QueryTreeNodePtr & join_tree_node)
{
const JoinNode * join_node = join_tree_node->as<JoinNode>();
if (!join_node)
return true;
const auto join_kind = join_node->getKind();
if (join_kind == JoinKind::Left || join_kind == JoinKind::Right || join_kind == JoinKind::Inner)
return true;
return false;
}(parent_join_tree);
if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0) if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0)
{ {
if (auto cluster = query_context->getClusterForParallelReplicas(); if (auto cluster = query_context->getClusterForParallelReplicas();
@ -964,7 +990,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
query_plan = std::move(query_plan_parallel_replicas); query_plan = std::move(query_plan_parallel_replicas);
} }
} }
else if (ClusterProxy::canUseParallelReplicasOnInitiator(query_context)) else if (ClusterProxy::canUseParallelReplicasOnInitiator(query_context) && allow_parallel_replicas_for_table_expression)
{ {
// (1) find read step // (1) find read step
QueryPlan::Node * node = query_plan.getRootNode(); QueryPlan::Node * node = query_plan.getRootNode();
@ -1794,7 +1820,8 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
const ColumnIdentifierSet & outer_scope_columns, const ColumnIdentifierSet & outer_scope_columns,
PlannerContextPtr & planner_context) PlannerContextPtr & planner_context)
{ {
auto table_expressions_stack = buildTableExpressionsStack(query_node->as<QueryNode &>().getJoinTree()); const QueryTreeNodePtr & join_tree_node = query_node->as<QueryNode &>().getJoinTree();
auto table_expressions_stack = buildTableExpressionsStack(join_tree_node);
size_t table_expressions_stack_size = table_expressions_stack.size(); size_t table_expressions_stack_size = table_expressions_stack.size();
bool is_single_table_expression = table_expressions_stack_size == 1; bool is_single_table_expression = table_expressions_stack_size == 1;
@ -1829,7 +1856,9 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
* Examples: Distributed, LiveView, Merge storages. * Examples: Distributed, LiveView, Merge storages.
*/ */
auto left_table_expression = table_expressions_stack.front(); auto left_table_expression = table_expressions_stack.front();
auto left_table_expression_query_plan = buildQueryPlanForTableExpression(left_table_expression, auto left_table_expression_query_plan = buildQueryPlanForTableExpression(
left_table_expression,
join_tree_node,
select_query_info, select_query_info,
select_query_options, select_query_options,
planner_context, planner_context,
@ -1902,7 +1931,9 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
* table expression in subquery. * table expression in subquery.
*/ */
bool is_remote = planner_context->getTableExpressionDataOrThrow(table_expression).isRemote(); bool is_remote = planner_context->getTableExpressionDataOrThrow(table_expression).isRemote();
query_plans_stack.push_back(buildQueryPlanForTableExpression(table_expression, query_plans_stack.push_back(buildQueryPlanForTableExpression(
table_expression,
join_tree_node,
select_query_info, select_query_info,
select_query_options, select_query_options,
planner_context, planner_context,

View File

@ -23,6 +23,8 @@
#include <Storages/StorageMaterializedView.h> #include <Storages/StorageMaterializedView.h>
#include <Storages/buildQueryTreeForShard.h> #include <Storages/buildQueryTreeForShard.h>
#include <ranges>
namespace DB namespace DB
{ {
namespace Setting namespace Setting
@ -38,12 +40,12 @@ namespace ErrorCodes
/// Returns a list of (sub)queries (candidates) which may support parallel replicas. /// Returns a list of (sub)queries (candidates) which may support parallel replicas.
/// The rule is : /// The rule is :
/// subquery has only LEFT or ALL INNER JOIN (or none), and left part is MergeTree table or subquery candidate as well. /// subquery has only LEFT / RIGHT / ALL INNER JOIN (or none), and left / right part is MergeTree table or subquery candidate as well.
/// ///
/// Additional checks are required, so we return many candidates. The innermost subquery is on top. /// Additional checks are required, so we return many candidates. The innermost subquery is on top.
std::stack<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTreeNode * query_tree_node) std::vector<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTreeNode * query_tree_node)
{ {
std::stack<const QueryNode *> res; std::vector<const QueryNode *> res;
while (query_tree_node) while (query_tree_node)
{ {
@ -75,7 +77,7 @@ std::stack<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTre
{ {
const auto & query_node_to_process = query_tree_node->as<QueryNode &>(); const auto & query_node_to_process = query_tree_node->as<QueryNode &>();
query_tree_node = query_node_to_process.getJoinTree().get(); query_tree_node = query_node_to_process.getJoinTree().get();
res.push(&query_node_to_process); res.push_back(&query_node_to_process);
break; break;
} }
case QueryTreeNodeType::UNION: case QueryTreeNodeType::UNION:
@ -98,17 +100,16 @@ std::stack<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTre
case QueryTreeNodeType::JOIN: case QueryTreeNodeType::JOIN:
{ {
const auto & join_node = query_tree_node->as<JoinNode &>(); const auto & join_node = query_tree_node->as<JoinNode &>();
auto join_kind = join_node.getKind(); const auto join_kind = join_node.getKind();
auto join_strictness = join_node.getStrictness(); const auto join_strictness = join_node.getStrictness();
bool can_parallelize_join = if (join_kind == JoinKind::Left || (join_kind == JoinKind::Inner && join_strictness == JoinStrictness::All))
join_kind == JoinKind::Left query_tree_node = join_node.getLeftTableExpression().get();
|| (join_kind == JoinKind::Inner && join_strictness == JoinStrictness::All); else if (join_kind == JoinKind::Right)
query_tree_node = join_node.getRightTableExpression().get();
if (!can_parallelize_join) else
return {}; return {};
query_tree_node = join_node.getLeftTableExpression().get();
break; break;
} }
default: default:
@ -163,14 +164,27 @@ QueryTreeNodePtr replaceTablesWithDummyTables(QueryTreeNodePtr query, const Cont
return query->cloneAndReplace(visitor.replacement_map); return query->cloneAndReplace(visitor.replacement_map);
} }
#ifdef DUMP_PARALLEL_REPLICAS_QUERY_CANDIDATES
static void dumpStack(const std::vector<const QueryNode *> & stack)
{
std::ranges::reverse_view rv{stack};
for (const auto * node : rv)
LOG_DEBUG(getLogger(__PRETTY_FUNCTION__), "{}\n{}", CityHash_v1_0_2::Hash128to64(node->getTreeHash()), node->dumpTree());
}
#endif
/// Find the best candidate for parallel replicas execution by verifying query plan. /// Find the best candidate for parallel replicas execution by verifying query plan.
/// If query plan has only Expression, Filter of Join steps, we can execute it fully remotely and check the next query. /// If query plan has only Expression, Filter or Join steps, we can execute it fully remotely and check the next query.
/// Otherwise we can execute current query up to WithMergableStage only. /// Otherwise we can execute current query up to WithMergableStage only.
const QueryNode * findQueryForParallelReplicas( const QueryNode * findQueryForParallelReplicas(
std::stack<const QueryNode *> stack, std::vector<const QueryNode *> stack,
const std::unordered_map<const QueryNode *, const QueryPlan::Node *> & mapping, const std::unordered_map<const QueryNode *, const QueryPlan::Node *> & mapping,
const Settings & settings) const Settings & settings)
{ {
#ifdef DUMP_PARALLEL_REPLICAS_QUERY_CANDIDATES
dumpStack(stack);
#endif
struct Frame struct Frame
{ {
const QueryPlan::Node * node = nullptr; const QueryPlan::Node * node = nullptr;
@ -189,8 +203,8 @@ const QueryNode * findQueryForParallelReplicas(
while (!stack.empty()) while (!stack.empty())
{ {
const QueryNode * const subquery_node = stack.top(); const QueryNode * const subquery_node = stack.back();
stack.pop(); stack.pop_back();
auto it = mapping.find(subquery_node); auto it = mapping.find(subquery_node);
/// This should not happen ideally. /// This should not happen ideally.
@ -236,7 +250,7 @@ const QueryNode * findQueryForParallelReplicas(
else else
{ {
const auto * join = typeid_cast<JoinStep *>(step); const auto * join = typeid_cast<JoinStep *>(step);
/// We've checked that JOIN is INNER/LEFT in query tree. /// We've checked that JOIN is INNER/LEFT/RIGHT on query tree level before.
/// Don't distribute UNION node. /// Don't distribute UNION node.
if (!join) if (!join)
return res; return res;
@ -263,7 +277,7 @@ const QueryNode * findQueryForParallelReplicas(
return res; return res;
} }
const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options) const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tree_node, const SelectQueryOptions & select_query_options)
{ {
if (select_query_options.only_analyze) if (select_query_options.only_analyze)
return nullptr; return nullptr;
@ -287,7 +301,7 @@ const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tr
return nullptr; return nullptr;
/// We don't have any subquery and storage can process parallel replicas by itself. /// We don't have any subquery and storage can process parallel replicas by itself.
if (stack.top() == query_tree_node.get()) if (stack.back() == query_tree_node.get())
return nullptr; return nullptr;
/// This is needed to avoid infinite recursion. /// This is needed to avoid infinite recursion.
@ -310,31 +324,33 @@ const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tr
const auto & mapping = planner.getQueryNodeToPlanStepMapping(); const auto & mapping = planner.getQueryNodeToPlanStepMapping();
const auto * res = findQueryForParallelReplicas(new_stack, mapping, context->getSettingsRef()); const auto * res = findQueryForParallelReplicas(new_stack, mapping, context->getSettingsRef());
/// Now, return a query from initial stack.
if (res) if (res)
{ {
// find query in initial stack
while (!new_stack.empty()) while (!new_stack.empty())
{ {
if (res == new_stack.top()) if (res == new_stack.back())
return stack.top(); {
res = stack.back();
break;
}
stack.pop(); stack.pop_back();
new_stack.pop(); new_stack.pop_back();
} }
} }
return res; return res;
} }
static const TableNode * findTableForParallelReplicas(const IQueryTreeNode * query_tree_node) static const TableNode * findTableForParallelReplicas(const IQueryTreeNode * query_tree_node)
{ {
std::stack<const IQueryTreeNode *> right_join_nodes; std::stack<const IQueryTreeNode *> join_nodes;
while (query_tree_node || !right_join_nodes.empty()) while (query_tree_node || !join_nodes.empty())
{ {
if (!query_tree_node) if (!query_tree_node)
{ {
query_tree_node = right_join_nodes.top(); query_tree_node = join_nodes.top();
right_join_nodes.pop(); join_nodes.pop();
} }
auto join_tree_node_type = query_tree_node->getNodeType(); auto join_tree_node_type = query_tree_node->getNodeType();
@ -383,8 +399,23 @@ static const TableNode * findTableForParallelReplicas(const IQueryTreeNode * que
case QueryTreeNodeType::JOIN: case QueryTreeNodeType::JOIN:
{ {
const auto & join_node = query_tree_node->as<JoinNode &>(); const auto & join_node = query_tree_node->as<JoinNode &>();
query_tree_node = join_node.getLeftTableExpression().get(); const auto join_kind = join_node.getKind();
right_join_nodes.push(join_node.getRightTableExpression().get()); const auto join_strictness = join_node.getStrictness();
if (join_kind == JoinKind::Left || (join_kind == JoinKind::Inner and join_strictness == JoinStrictness::All))
{
query_tree_node = join_node.getLeftTableExpression().get();
join_nodes.push(join_node.getRightTableExpression().get());
}
else if (join_kind == JoinKind::Right)
{
query_tree_node = join_node.getRightTableExpression().get();
join_nodes.push(join_node.getLeftTableExpression().get());
}
else
{
return nullptr;
}
break; break;
} }
default: default:
@ -400,7 +431,7 @@ static const TableNode * findTableForParallelReplicas(const IQueryTreeNode * que
return nullptr; return nullptr;
} }
const TableNode * findTableForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options) const TableNode * findTableForParallelReplicas(const QueryTreeNodePtr & query_tree_node, const SelectQueryOptions & select_query_options)
{ {
if (select_query_options.only_analyze) if (select_query_options.only_analyze)
return nullptr; return nullptr;

View File

@ -15,10 +15,10 @@ struct SelectQueryOptions;
/// Find a query which can be executed with parallel replicas up to WithMergableStage. /// Find a query which can be executed with parallel replicas up to WithMergableStage.
/// Returned query will always contain some (>1) subqueries, possibly with joins. /// Returned query will always contain some (>1) subqueries, possibly with joins.
const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options); const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tree_node, const SelectQueryOptions & select_query_options);
/// Find a table from which we should read on follower replica. It's the left-most table within all JOINs and UNIONs. /// Find a table from which we should read on follower replica. It's the left-most table within all JOINs and UNIONs.
const TableNode * findTableForParallelReplicas(const QueryTreeNodePtr & query_tree_node, SelectQueryOptions & select_query_options); const TableNode * findTableForParallelReplicas(const QueryTreeNodePtr & query_tree_node, const SelectQueryOptions & select_query_options);
struct JoinTreeQueryPlan; struct JoinTreeQueryPlan;

View File

@ -79,7 +79,7 @@ bool ExecutionThreadContext::executeTask()
if (trace_processors) if (trace_processors)
{ {
span = std::make_unique<OpenTelemetry::SpanHolder>(node->processor->getName()); span = std::make_unique<OpenTelemetry::SpanHolder>(node->processor->getUniqID());
span->addAttribute("thread_number", thread_number); span->addAttribute("thread_number", thread_number);
} }
std::optional<Stopwatch> execution_time_watch; std::optional<Stopwatch> execution_time_watch;

View File

@ -10,6 +10,20 @@
namespace DB namespace DB
{ {
IProcessor::IProcessor()
{
processor_index = CurrentThread::isInitialized() ? CurrentThread::get().getNextPipelineProcessorIndex() : 0;
}
IProcessor::IProcessor(InputPorts inputs_, OutputPorts outputs_) : inputs(std::move(inputs_)), outputs(std::move(outputs_))
{
for (auto & port : inputs)
port.processor = this;
for (auto & port : outputs)
port.processor = this;
processor_index = CurrentThread::isInitialized() ? CurrentThread::get().getNextPipelineProcessorIndex() : 0;
}
void IProcessor::setQueryPlanStep(IQueryPlanStep * step, size_t group) void IProcessor::setQueryPlanStep(IQueryPlanStep * step, size_t group)
{ {
query_plan_step = step; query_plan_step = step;
@ -18,6 +32,7 @@ void IProcessor::setQueryPlanStep(IQueryPlanStep * step, size_t group)
{ {
plan_step_name = step->getName(); plan_step_name = step->getName();
plan_step_description = step->getStepDescription(); plan_step_description = step->getStepDescription();
step_uniq_id = step->getUniqID();
} }
} }

View File

@ -1,9 +1,12 @@
#pragma once #pragma once
#include <memory> #include <Interpreters/Context.h>
#include <Processors/Port.h> #include <Processors/Port.h>
#include <Processors/QueryPlan/IQueryPlanStep.h>
#include <Common/CurrentThread.h>
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
#include <memory>
class EventCounter; class EventCounter;
@ -121,19 +124,14 @@ protected:
OutputPorts outputs; OutputPorts outputs;
public: public:
IProcessor() = default; IProcessor();
IProcessor(InputPorts inputs_, OutputPorts outputs_) IProcessor(InputPorts inputs_, OutputPorts outputs_);
: inputs(std::move(inputs_)), outputs(std::move(outputs_))
{
for (auto & port : inputs)
port.processor = this;
for (auto & port : outputs)
port.processor = this;
}
virtual String getName() const = 0; virtual String getName() const = 0;
String getUniqID() const { return fmt::format("{}_{}", getName(), processor_index); }
enum class Status : uint8_t enum class Status : uint8_t
{ {
/// Processor needs some data at its inputs to proceed. /// Processor needs some data at its inputs to proceed.
@ -314,6 +312,7 @@ public:
void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0); void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0);
IQueryPlanStep * getQueryPlanStep() const { return query_plan_step; } IQueryPlanStep * getQueryPlanStep() const { return query_plan_step; }
const String & getStepUniqID() const { return step_uniq_id; }
size_t getQueryPlanStepGroup() const { return query_plan_step_group; } size_t getQueryPlanStepGroup() const { return query_plan_step_group; }
const String & getPlanStepName() const { return plan_step_name; } const String & getPlanStepName() const { return plan_step_name; }
const String & getPlanStepDescription() const { return plan_step_description; } const String & getPlanStepDescription() const { return plan_step_description; }
@ -407,7 +406,10 @@ private:
size_t stream_number = NO_STREAM; size_t stream_number = NO_STREAM;
IQueryPlanStep * query_plan_step = nullptr; IQueryPlanStep * query_plan_step = nullptr;
String step_uniq_id;
size_t query_plan_step_group = 0; size_t query_plan_step_group = 0;
size_t processor_index = 0;
String plan_step_name; String plan_step_name;
String plan_step_description; String plan_step_description;
}; };

View File

@ -5,6 +5,7 @@
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <Common/JSONBuilder.h> #include <Common/JSONBuilder.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeLowCardinality.h> #include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Functions/IFunction.h> #include <Functions/IFunction.h>
@ -52,7 +53,7 @@ static ActionsAndName splitSingleAndFilter(ActionsDAG & dag, const ActionsDAG::N
auto filter_type = removeLowCardinality(split_filter_node->result_type); auto filter_type = removeLowCardinality(split_filter_node->result_type);
if (!filter_type->onlyNull() && !isUInt8(removeNullable(filter_type))) if (!filter_type->onlyNull() && !isUInt8(removeNullable(filter_type)))
{ {
DataTypePtr cast_type = std::make_shared<DataTypeUInt8>(); DataTypePtr cast_type = DataTypeFactory::instance().get("Bool");
if (filter_type->isNullable()) if (filter_type->isNullable())
cast_type = std::make_shared<DataTypeNullable>(std::move(cast_type)); cast_type = std::make_shared<DataTypeNullable>(std::move(cast_type));

View File

@ -10,6 +10,11 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
IQueryPlanStep::IQueryPlanStep()
{
step_index = CurrentThread::isInitialized() ? CurrentThread::get().getNextPlanStepIndex() : 0;
}
void IQueryPlanStep::updateInputHeaders(Headers input_headers_) void IQueryPlanStep::updateInputHeaders(Headers input_headers_)
{ {
input_headers = std::move(input_headers_); input_headers = std::move(input_headers_);

View File

@ -1,8 +1,13 @@
#pragma once #pragma once
#include <Common/CurrentThread.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <Core/SortDescription.h> #include <Core/SortDescription.h>
#include <Interpreters/Context.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h> #include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <fmt/core.h>
namespace DB namespace DB
{ {
@ -26,6 +31,8 @@ using Headers = std::vector<Header>;
class IQueryPlanStep class IQueryPlanStep
{ {
public: public:
IQueryPlanStep();
virtual ~IQueryPlanStep() = default; virtual ~IQueryPlanStep() = default;
virtual String getName() const = 0; virtual String getName() const = 0;
@ -77,6 +84,8 @@ public:
/// Updates the input streams of the given step. Used during query plan optimizations. /// Updates the input streams of the given step. Used during query plan optimizations.
/// It won't do any validation of new streams, so it is your responsibility to ensure that this update doesn't break anything /// It won't do any validation of new streams, so it is your responsibility to ensure that this update doesn't break anything
String getUniqID() const { return fmt::format("{}_{}", getName(), step_index); }
/// (e.g. you correctly remove / add columns). /// (e.g. you correctly remove / add columns).
void updateInputHeaders(Headers input_headers_); void updateInputHeaders(Headers input_headers_);
void updateInputHeader(Header input_header, size_t idx = 0); void updateInputHeader(Header input_header, size_t idx = 0);
@ -95,6 +104,9 @@ protected:
Processors processors; Processors processors;
static void describePipeline(const Processors & processors, FormatSettings & settings); static void describePipeline(const Processors & processors, FormatSettings & settings);
private:
size_t step_index = 0;
}; };
using QueryPlanStepPtr = std::unique_ptr<IQueryPlanStep>; using QueryPlanStepPtr = std::unique_ptr<IQueryPlanStep>;

View File

@ -3,12 +3,15 @@
#include <Common/checkStackSize.h> #include <Common/checkStackSize.h>
#include <Interpreters/ActionsDAG.h> #include <Interpreters/ActionsDAG.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/IJoin.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h> #include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Interpreters/StorageID.h> #include <Interpreters/StorageID.h>
#include <Interpreters/TableJoin.h>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>
#include <Processors/QueryPlan/ConvertingActions.h> #include <Processors/QueryPlan/ConvertingActions.h>
#include <Processors/QueryPlan/ExpressionStep.h> #include <Processors/QueryPlan/ExpressionStep.h>
#include <Processors/QueryPlan/ISourceStep.h> #include <Processors/QueryPlan/ISourceStep.h>
#include <Processors/QueryPlan/JoinStep.h>
#include <Processors/QueryPlan/ReadFromMergeTree.h> #include <Processors/QueryPlan/ReadFromMergeTree.h>
#include <Processors/Sources/NullSource.h> #include <Processors/Sources/NullSource.h>
#include <Processors/Transforms/ExpressionTransform.h> #include <Processors/Transforms/ExpressionTransform.h>
@ -62,7 +65,14 @@ std::pair<std::unique_ptr<QueryPlan>, bool> createLocalPlanForParallelReplicas(
break; break;
if (!node->children.empty()) if (!node->children.empty())
node = node->children.at(0); {
// in case of RIGHT JOIN, - reading from right table is parallelized among replicas
const JoinStep * join = typeid_cast<JoinStep*>(node->step.get());
if (join && join->getJoin()->getTableJoin().kind() == JoinKind::Right)
node = node->children.at(1);
else
node = node->children.at(0);
}
else else
node = nullptr; node = nullptr;
} }

View File

@ -207,6 +207,7 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline(
static void explainStep(const IQueryPlanStep & step, JSONBuilder::JSONMap & map, const QueryPlan::ExplainPlanOptions & options) static void explainStep(const IQueryPlanStep & step, JSONBuilder::JSONMap & map, const QueryPlan::ExplainPlanOptions & options)
{ {
map.add("Node Type", step.getName()); map.add("Node Type", step.getName());
map.add("Node Id", step.getUniqID());
if (options.description) if (options.description)
{ {

View File

@ -282,9 +282,9 @@ void SortingStep::mergeSorting(
if (increase_sort_description_compile_attempts) if (increase_sort_description_compile_attempts)
increase_sort_description_compile_attempts = false; increase_sort_description_compile_attempts = false;
auto tmp_data_on_disk = sort_settings.tmp_data TemporaryDataOnDiskScopePtr tmp_data_on_disk = nullptr;
? std::make_unique<TemporaryDataOnDisk>(sort_settings.tmp_data, CurrentMetrics::TemporaryFilesForSort) if (sort_settings.tmp_data)
: std::unique_ptr<TemporaryDataOnDisk>(); tmp_data_on_disk = sort_settings.tmp_data->childScope(CurrentMetrics::TemporaryFilesForSort);
return std::make_shared<MergeSortingTransform>( return std::make_shared<MergeSortingTransform>(
header, header,

View File

@ -54,9 +54,9 @@ namespace
class SourceFromNativeStream : public ISource class SourceFromNativeStream : public ISource
{ {
public: public:
explicit SourceFromNativeStream(TemporaryFileStream * tmp_stream_) explicit SourceFromNativeStream(const Block & header, TemporaryBlockStreamReaderHolder tmp_stream_)
: ISource(tmp_stream_->getHeader()) : ISource(header)
, tmp_stream(tmp_stream_) , tmp_stream(std::move(tmp_stream_))
{} {}
String getName() const override { return "SourceFromNativeStream"; } String getName() const override { return "SourceFromNativeStream"; }
@ -69,7 +69,7 @@ namespace
auto block = tmp_stream->read(); auto block = tmp_stream->read();
if (!block) if (!block)
{ {
tmp_stream = nullptr; tmp_stream.reset();
return {}; return {};
} }
return convertToChunk(block); return convertToChunk(block);
@ -78,7 +78,7 @@ namespace
std::optional<ReadProgress> getReadProgress() override { return std::nullopt; } std::optional<ReadProgress> getReadProgress() override { return std::nullopt; }
private: private:
TemporaryFileStream * tmp_stream; TemporaryBlockStreamReaderHolder tmp_stream;
}; };
} }
@ -811,15 +811,18 @@ void AggregatingTransform::initGenerate()
Pipes pipes; Pipes pipes;
/// Merge external data from all aggregators used in query. /// Merge external data from all aggregators used in query.
for (const auto & aggregator : *params->aggregator_list_ptr) for (auto & aggregator : *params->aggregator_list_ptr)
{ {
const auto & tmp_data = aggregator.getTemporaryData(); tmp_files = aggregator.detachTemporaryData();
for (auto * tmp_stream : tmp_data.getStreams()) num_streams += tmp_files.size();
pipes.emplace_back(Pipe(std::make_unique<SourceFromNativeStream>(tmp_stream)));
num_streams += tmp_data.getStreams().size(); for (auto & tmp_stream : tmp_files)
compressed_size += tmp_data.getStat().compressed_size; {
uncompressed_size += tmp_data.getStat().uncompressed_size; auto stat = tmp_stream.finishWriting();
compressed_size += stat.compressed_size;
uncompressed_size += stat.uncompressed_size;
pipes.emplace_back(Pipe(std::make_unique<SourceFromNativeStream>(tmp_stream.getHeader(), tmp_stream.getReadStream())));
}
} }
LOG_DEBUG( LOG_DEBUG(

View File

@ -216,6 +216,8 @@ private:
RowsBeforeStepCounterPtr rows_before_aggregation; RowsBeforeStepCounterPtr rows_before_aggregation;
std::list<TemporaryBlockStreamHolder> tmp_files;
void initGenerate(); void initGenerate();
}; };

View File

@ -27,15 +27,20 @@ namespace ProfileEvents
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
class BufferingToFileTransform : public IAccumulatingTransform class BufferingToFileTransform : public IAccumulatingTransform
{ {
public: public:
BufferingToFileTransform(const Block & header, TemporaryFileStream & tmp_stream_, LoggerPtr log_) BufferingToFileTransform(const Block & header, TemporaryBlockStreamHolder tmp_stream_, LoggerPtr log_)
: IAccumulatingTransform(header, header) : IAccumulatingTransform(header, header)
, tmp_stream(tmp_stream_) , tmp_stream(std::move(tmp_stream_))
, log(log_) , log(log_)
{ {
LOG_INFO(log, "Sorting and writing part of data into temporary file {}", tmp_stream.getPath()); LOG_INFO(log, "Sorting and writing part of data into temporary file {}", tmp_stream.getHolder()->describeFilePath());
ProfileEvents::increment(ProfileEvents::ExternalSortWritePart); ProfileEvents::increment(ProfileEvents::ExternalSortWritePart);
} }
@ -44,14 +49,15 @@ public:
void consume(Chunk chunk) override void consume(Chunk chunk) override
{ {
Block block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns()); Block block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns());
tmp_stream.write(block); tmp_stream->write(block);
} }
Chunk generate() override Chunk generate() override
{ {
if (!tmp_stream.isWriteFinished()) if (!tmp_read_stream)
{ {
auto stat = tmp_stream.finishWriting(); auto stat = tmp_stream.finishWriting();
tmp_read_stream = tmp_stream.getReadStream();
ProfileEvents::increment(ProfileEvents::ExternalProcessingCompressedBytesTotal, stat.compressed_size); ProfileEvents::increment(ProfileEvents::ExternalProcessingCompressedBytesTotal, stat.compressed_size);
ProfileEvents::increment(ProfileEvents::ExternalProcessingUncompressedBytesTotal, stat.uncompressed_size); ProfileEvents::increment(ProfileEvents::ExternalProcessingUncompressedBytesTotal, stat.uncompressed_size);
@ -59,10 +65,11 @@ public:
ProfileEvents::increment(ProfileEvents::ExternalSortUncompressedBytes, stat.uncompressed_size); ProfileEvents::increment(ProfileEvents::ExternalSortUncompressedBytes, stat.uncompressed_size);
LOG_INFO(log, "Done writing part of data into temporary file {}, compressed {}, uncompressed {} ", LOG_INFO(log, "Done writing part of data into temporary file {}, compressed {}, uncompressed {} ",
tmp_stream.getPath(), ReadableSize(static_cast<double>(stat.compressed_size)), ReadableSize(static_cast<double>(stat.uncompressed_size))); tmp_stream.getHolder()->describeFilePath(),
ReadableSize(static_cast<double>(stat.compressed_size)), ReadableSize(static_cast<double>(stat.uncompressed_size)));
} }
Block block = tmp_stream.read(); Block block = tmp_read_stream.value()->read();
if (!block) if (!block)
return {}; return {};
@ -71,7 +78,8 @@ public:
} }
private: private:
TemporaryFileStream & tmp_stream; TemporaryBlockStreamHolder tmp_stream;
std::optional<TemporaryBlockStreamReaderHolder> tmp_read_stream;
LoggerPtr log; LoggerPtr log;
}; };
@ -86,7 +94,7 @@ MergeSortingTransform::MergeSortingTransform(
size_t max_bytes_before_remerge_, size_t max_bytes_before_remerge_,
double remerge_lowered_memory_bytes_ratio_, double remerge_lowered_memory_bytes_ratio_,
size_t max_bytes_before_external_sort_, size_t max_bytes_before_external_sort_,
TemporaryDataOnDiskPtr tmp_data_, TemporaryDataOnDiskScopePtr tmp_data_,
size_t min_free_disk_space_) size_t min_free_disk_space_)
: SortingTransform(header, description_, max_merged_block_size_, limit_, increase_sort_description_compile_attempts) : SortingTransform(header, description_, max_merged_block_size_, limit_, increase_sort_description_compile_attempts)
, max_bytes_before_remerge(max_bytes_before_remerge_) , max_bytes_before_remerge(max_bytes_before_remerge_)
@ -168,9 +176,13 @@ void MergeSortingTransform::consume(Chunk chunk)
*/ */
if (max_bytes_before_external_sort && sum_bytes_in_blocks > max_bytes_before_external_sort) if (max_bytes_before_external_sort && sum_bytes_in_blocks > max_bytes_before_external_sort)
{ {
if (!tmp_data)
throw Exception(ErrorCodes::LOGICAL_ERROR, "TemporaryDataOnDisk is not set for MergeSortingTransform");
temporary_files_num++;
/// If there's less free disk space than reserve_size, an exception will be thrown /// If there's less free disk space than reserve_size, an exception will be thrown
size_t reserve_size = sum_bytes_in_blocks + min_free_disk_space; size_t reserve_size = sum_bytes_in_blocks + min_free_disk_space;
auto & tmp_stream = tmp_data->createStream(header_without_constants, reserve_size); TemporaryBlockStreamHolder tmp_stream(header_without_constants, tmp_data.get(), reserve_size);
size_t max_merged_block_size = this->max_merged_block_size; size_t max_merged_block_size = this->max_merged_block_size;
if (max_block_bytes > 0 && sum_rows_in_blocks > 0 && sum_bytes_in_blocks > 0) if (max_block_bytes > 0 && sum_rows_in_blocks > 0 && sum_bytes_in_blocks > 0)
{ {
@ -179,7 +191,7 @@ void MergeSortingTransform::consume(Chunk chunk)
max_merged_block_size = std::max(std::min(max_merged_block_size, max_block_bytes / avg_row_bytes), 128UL); max_merged_block_size = std::max(std::min(max_merged_block_size, max_block_bytes / avg_row_bytes), 128UL);
} }
merge_sorter = std::make_unique<MergeSorter>(header_without_constants, std::move(chunks), description, max_merged_block_size, limit); merge_sorter = std::make_unique<MergeSorter>(header_without_constants, std::move(chunks), description, max_merged_block_size, limit);
auto current_processor = std::make_shared<BufferingToFileTransform>(header_without_constants, tmp_stream, log); auto current_processor = std::make_shared<BufferingToFileTransform>(header_without_constants, std::move(tmp_stream), log);
processors.emplace_back(current_processor); processors.emplace_back(current_processor);
@ -223,14 +235,14 @@ void MergeSortingTransform::generate()
{ {
if (!generated_prefix) if (!generated_prefix)
{ {
size_t num_tmp_files = tmp_data ? tmp_data->getStreams().size() : 0; if (temporary_files_num == 0)
if (num_tmp_files == 0) {
merge_sorter merge_sorter = std::make_unique<MergeSorter>(header_without_constants, std::move(chunks), description, max_merged_block_size, limit);
= std::make_unique<MergeSorter>(header_without_constants, std::move(chunks), description, max_merged_block_size, limit); }
else else
{ {
ProfileEvents::increment(ProfileEvents::ExternalSortMerge); ProfileEvents::increment(ProfileEvents::ExternalSortMerge);
LOG_INFO(log, "There are {} temporary sorted parts to merge", num_tmp_files); LOG_INFO(log, "There are {} temporary sorted parts to merge", temporary_files_num);
processors.emplace_back(std::make_shared<MergeSorterSource>( processors.emplace_back(std::make_shared<MergeSorterSource>(
header_without_constants, std::move(chunks), description, max_merged_block_size, limit)); header_without_constants, std::move(chunks), description, max_merged_block_size, limit));

View File

@ -29,7 +29,7 @@ public:
size_t max_bytes_before_remerge_, size_t max_bytes_before_remerge_,
double remerge_lowered_memory_bytes_ratio_, double remerge_lowered_memory_bytes_ratio_,
size_t max_bytes_before_external_sort_, size_t max_bytes_before_external_sort_,
TemporaryDataOnDiskPtr tmp_data_, TemporaryDataOnDiskScopePtr tmp_data_,
size_t min_free_disk_space_); size_t min_free_disk_space_);
String getName() const override { return "MergeSortingTransform"; } String getName() const override { return "MergeSortingTransform"; }
@ -45,7 +45,8 @@ private:
size_t max_bytes_before_remerge; size_t max_bytes_before_remerge;
double remerge_lowered_memory_bytes_ratio; double remerge_lowered_memory_bytes_ratio;
size_t max_bytes_before_external_sort; size_t max_bytes_before_external_sort;
TemporaryDataOnDiskPtr tmp_data; TemporaryDataOnDiskScopePtr tmp_data;
size_t temporary_files_num = 0;
size_t min_free_disk_space; size_t min_free_disk_space;
size_t max_block_bytes; size_t max_block_bytes;

View File

@ -398,10 +398,10 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
left->pipe.collected_processors = collected_processors; left->pipe.collected_processors = collected_processors;
/// Collect the NEW processors for the right pipeline.
QueryPipelineProcessorsCollector collector(*right);
/// Remember the last step of the right pipeline. /// Remember the last step of the right pipeline.
IQueryPlanStep * step = right->pipe.processors->back()->getQueryPlanStep(); IQueryPlanStep * step = right->pipe.processors->back()->getQueryPlanStep();
/// Collect the NEW processors for the right pipeline.
QueryPipelineProcessorsCollector collector(*right, step);
/// In case joined subquery has totals, and we don't, add default chunk to totals. /// In case joined subquery has totals, and we don't, add default chunk to totals.
bool default_totals = false; bool default_totals = false;

View File

@ -30,7 +30,7 @@ void printPipeline(const Processors & processors, const Statuses & statuses, Wri
for (const auto & processor : processors) for (const auto & processor : processors)
{ {
const auto & description = processor->getDescription(); const auto & description = processor->getDescription();
out << " n" << get_proc_id(*processor) << "[label=\"" << processor->getName() << (description.empty() ? "" : ":") << description; out << " n" << get_proc_id(*processor) << "[label=\"" << processor->getUniqID() << (description.empty() ? "" : ":") << description;
if (statuses_iter != statuses.end()) if (statuses_iter != statuses.end())
{ {

View File

@ -170,15 +170,16 @@ void HTTPHandler::pushDelayedResults(Output & used_output)
for (auto & write_buf : write_buffers) for (auto & write_buf : write_buffers)
{ {
if (!write_buf) if (auto * write_buf_concrete = dynamic_cast<TemporaryDataBuffer *>(write_buf.get()))
continue;
IReadableWriteBuffer * write_buf_concrete = dynamic_cast<IReadableWriteBuffer *>(write_buf.get());
if (write_buf_concrete)
{ {
ReadBufferPtr reread_buf = write_buf_concrete->tryGetReadBuffer(); if (auto reread_buf = write_buf_concrete->read())
if (reread_buf) read_buffers.emplace_back(std::move(reread_buf));
read_buffers.emplace_back(wrapReadBufferPointer(reread_buf)); }
if (auto * write_buf_concrete = dynamic_cast<IReadableWriteBuffer *>(write_buf.get()))
{
if (auto reread_buf = write_buf_concrete->tryGetReadBuffer())
read_buffers.emplace_back(std::move(reread_buf));
} }
} }
@ -321,21 +322,19 @@ void HTTPHandler::processQuery(
if (buffer_size_memory > 0 || buffer_until_eof) if (buffer_size_memory > 0 || buffer_until_eof)
{ {
CascadeWriteBuffer::WriteBufferPtrs cascade_buffer1; CascadeWriteBuffer::WriteBufferPtrs cascade_buffers;
CascadeWriteBuffer::WriteBufferConstructors cascade_buffer2; CascadeWriteBuffer::WriteBufferConstructors cascade_buffers_lazy;
if (buffer_size_memory > 0) if (buffer_size_memory > 0)
cascade_buffer1.emplace_back(std::make_shared<MemoryWriteBuffer>(buffer_size_memory)); cascade_buffers.emplace_back(std::make_shared<MemoryWriteBuffer>(buffer_size_memory));
if (buffer_until_eof) if (buffer_until_eof)
{ {
auto tmp_data = std::make_shared<TemporaryDataOnDisk>(server.context()->getTempDataOnDisk()); auto tmp_data = server.context()->getTempDataOnDisk();
cascade_buffers_lazy.emplace_back([tmp_data](const WriteBufferPtr &) -> WriteBufferPtr
auto create_tmp_disk_buffer = [tmp_data] (const WriteBufferPtr &) -> WriteBufferPtr { {
return tmp_data->createRawStream(); return std::make_unique<TemporaryDataBuffer>(tmp_data.get());
}; });
cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer));
} }
else else
{ {
@ -351,10 +350,10 @@ void HTTPHandler::processQuery(
return next_buffer; return next_buffer;
}; };
cascade_buffer2.emplace_back(push_memory_buffer_and_continue); cascade_buffers_lazy.emplace_back(push_memory_buffer_and_continue);
} }
used_output.out_delayed_and_compressed_holder = std::make_unique<CascadeWriteBuffer>(std::move(cascade_buffer1), std::move(cascade_buffer2)); used_output.out_delayed_and_compressed_holder = std::make_unique<CascadeWriteBuffer>(std::move(cascade_buffers), std::move(cascade_buffers_lazy));
used_output.out_maybe_delayed_and_compressed = used_output.out_delayed_and_compressed_holder.get(); used_output.out_maybe_delayed_and_compressed = used_output.out_delayed_and_compressed_holder.get();
} }
else else

View File

@ -65,6 +65,11 @@ namespace ProfileEvents
extern const Event MergeProjectionStageExecuteMilliseconds; extern const Event MergeProjectionStageExecuteMilliseconds;
} }
namespace CurrentMetrics
{
extern const Metric TemporaryFilesForMerge;
}
namespace DB namespace DB
{ {
namespace Setting namespace Setting
@ -124,6 +129,7 @@ static ColumnsStatistics getStatisticsForColumns(
return all_statistics; return all_statistics;
} }
/// Manages the "rows_sources" temporary file that is used during vertical merge. /// Manages the "rows_sources" temporary file that is used during vertical merge.
class RowsSourcesTemporaryFile : public ITemporaryFileLookup class RowsSourcesTemporaryFile : public ITemporaryFileLookup
{ {
@ -132,9 +138,7 @@ public:
static constexpr auto FILE_ID = "rows_sources"; static constexpr auto FILE_ID = "rows_sources";
explicit RowsSourcesTemporaryFile(TemporaryDataOnDiskScopePtr temporary_data_on_disk_) explicit RowsSourcesTemporaryFile(TemporaryDataOnDiskScopePtr temporary_data_on_disk_)
: tmp_disk(std::make_unique<TemporaryDataOnDisk>(temporary_data_on_disk_)) : temporary_data_on_disk(temporary_data_on_disk_->childScope(CurrentMetrics::TemporaryFilesForMerge))
, uncompressed_write_buffer(tmp_disk->createRawStream())
, tmp_file_name_on_disk(uncompressed_write_buffer->getFileName())
{ {
} }
@ -143,11 +147,11 @@ public:
if (name != FILE_ID) if (name != FILE_ID)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected temporary file name requested: {}", name); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected temporary file name requested: {}", name);
if (write_buffer) if (tmp_data_buffer)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file was already requested for writing, there musto be only one writer"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file was already requested for writing, there musto be only one writer");
write_buffer = (std::make_unique<CompressedWriteBuffer>(*uncompressed_write_buffer)); tmp_data_buffer = std::make_unique<TemporaryDataBuffer>(temporary_data_on_disk.get());
return *write_buffer; return *tmp_data_buffer;
} }
std::unique_ptr<ReadBuffer> getTemporaryFileForReading(const String & name) override std::unique_ptr<ReadBuffer> getTemporaryFileForReading(const String & name) override
@ -163,25 +167,24 @@ public:
return std::make_unique<ReadBufferFromEmptyFile>(); return std::make_unique<ReadBufferFromEmptyFile>();
/// Reopen the file for each read so that multiple reads can be performed in parallel and there is no need to seek to the beginning. /// Reopen the file for each read so that multiple reads can be performed in parallel and there is no need to seek to the beginning.
auto raw_file_read_buffer = std::make_unique<ReadBufferFromFile>(tmp_file_name_on_disk); return tmp_data_buffer->read();
return std::make_unique<CompressedReadBufferFromFile>(std::move(raw_file_read_buffer));
} }
/// Returns written data size in bytes /// Returns written data size in bytes
size_t finalizeWriting() size_t finalizeWriting()
{ {
write_buffer->finalize(); if (!tmp_data_buffer)
uncompressed_write_buffer->finalize(); throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file was not requested for writing");
auto stat = tmp_data_buffer->finishWriting();
finalized = true; finalized = true;
final_size = write_buffer->count(); final_size = stat.uncompressed_size;
return final_size; return final_size;
} }
private: private:
std::unique_ptr<TemporaryDataOnDisk> tmp_disk; std::unique_ptr<TemporaryDataBuffer> tmp_data_buffer;
std::unique_ptr<WriteBufferFromFileBase> uncompressed_write_buffer; TemporaryDataOnDiskScopePtr temporary_data_on_disk;
std::unique_ptr<WriteBuffer> write_buffer;
const String tmp_file_name_on_disk;
bool finalized = false; bool finalized = false;
size_t final_size = 0; size_t final_size = 0;
}; };
@ -874,6 +877,7 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const
/// In special case, when there is only one source part, and no rows were skipped, we may have /// In special case, when there is only one source part, and no rows were skipped, we may have
/// skipped writing rows_sources file. Otherwise rows_sources_count must be equal to the total /// skipped writing rows_sources file. Otherwise rows_sources_count must be equal to the total
/// number of input rows. /// number of input rows.
/// Note that only one byte index is written for each row, so number of rows is equals to the number of bytes written.
if ((rows_sources_count > 0 || global_ctx->future_part->parts.size() > 1) && sum_input_rows_exact != rows_sources_count + input_rows_filtered) if ((rows_sources_count > 0 || global_ctx->future_part->parts.size() > 1) && sum_input_rows_exact != rows_sources_count + input_rows_filtered)
throw Exception( throw Exception(
ErrorCodes::LOGICAL_ERROR, ErrorCodes::LOGICAL_ERROR,
@ -881,6 +885,7 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const
"of bytes written to rows_sources file ({}). It is a bug.", "of bytes written to rows_sources file ({}). It is a bug.",
sum_input_rows_exact, input_rows_filtered, rows_sources_count); sum_input_rows_exact, input_rows_filtered, rows_sources_count);
ctx->it_name_and_type = global_ctx->gathering_columns.cbegin(); ctx->it_name_and_type = global_ctx->gathering_columns.cbegin();
const auto & settings = global_ctx->context->getSettingsRef(); const auto & settings = global_ctx->context->getSettingsRef();
@ -1718,7 +1723,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const
sort_description, sort_description,
partition_key_columns, partition_key_columns,
global_ctx->merging_params, global_ctx->merging_params,
(is_vertical_merge ? RowsSourcesTemporaryFile::FILE_ID : ""), /// rows_sources' temporary file is used only for vertical merge (is_vertical_merge ? RowsSourcesTemporaryFile::FILE_ID : ""), /// rows_sources' temporary file is used only for vertical merge
(*data_settings)[MergeTreeSetting::merge_max_block_size], (*data_settings)[MergeTreeSetting::merge_max_block_size],
(*data_settings)[MergeTreeSetting::merge_max_block_size_bytes], (*data_settings)[MergeTreeSetting::merge_max_block_size_bytes],
ctx->blocks_are_granules_size, ctx->blocks_are_granules_size,

View File

@ -162,8 +162,6 @@ struct SelectQueryInfo
/// It's guaranteed to be present in JOIN TREE of `query_tree` /// It's guaranteed to be present in JOIN TREE of `query_tree`
QueryTreeNodePtr table_expression; QueryTreeNodePtr table_expression;
bool current_table_chosen_for_reading_with_parallel_replicas = false;
/// Table expression modifiers for storage /// Table expression modifiers for storage
std::optional<TableExpressionModifiers> table_expression_modifiers; std::optional<TableExpressionModifiers> table_expression_modifiers;

View File

@ -276,9 +276,7 @@ void StorageMergeTree::read(
} }
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower()
&& local_context->getSettingsRef()[Setting::parallel_replicas_for_non_replicated_merge_tree] && local_context->getSettingsRef()[Setting::parallel_replicas_for_non_replicated_merge_tree];
&& (!local_context->getSettingsRef()[Setting::allow_experimental_analyzer]
|| query_info.current_table_chosen_for_reading_with_parallel_replicas);
if (auto plan = reader.read( if (auto plan = reader.read(
column_names, column_names,

View File

@ -5640,10 +5640,7 @@ void StorageReplicatedMergeTree::readLocalImpl(
const size_t max_block_size, const size_t max_block_size,
const size_t num_streams) const size_t num_streams)
{ {
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower() const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower();
&& (!local_context->getSettingsRef()[Setting::allow_experimental_analyzer]
|| query_info.current_table_chosen_for_reading_with_parallel_replicas);
auto plan = reader.read( auto plan = reader.read(
column_names, storage_snapshot, query_info, column_names, storage_snapshot, query_info,
local_context, max_block_size, num_streams, local_context, max_block_size, num_streams,

View File

@ -314,6 +314,35 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_node,
return temporary_table_expression_node; return temporary_table_expression_node;
} }
QueryTreeNodePtr getSubqueryFromTableExpression(
const QueryTreeNodePtr & join_table_expression,
const std::unordered_map<QueryTreeNodePtr, CollectColumnSourceToColumnsVisitor::Columns> & column_source_to_columns,
const ContextPtr & context)
{
auto join_table_expression_node_type = join_table_expression->getNodeType();
QueryTreeNodePtr subquery_node;
if (join_table_expression_node_type == QueryTreeNodeType::QUERY || join_table_expression_node_type == QueryTreeNodeType::UNION)
{
subquery_node = join_table_expression;
}
else if (
join_table_expression_node_type == QueryTreeNodeType::TABLE || join_table_expression_node_type == QueryTreeNodeType::TABLE_FUNCTION)
{
const auto & columns = column_source_to_columns.at(join_table_expression).columns;
subquery_node = buildSubqueryToReadColumnsFromTableExpression(columns, join_table_expression, context);
}
else
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Expected JOIN table expression to be table, table function, query or union node. Actual {}",
join_table_expression->formatASTForErrorMessage());
}
return subquery_node;
}
} }
QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_context, QueryTreeNodePtr query_tree_to_modify) QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_context, QueryTreeNodePtr query_tree_to_modify)
@ -335,37 +364,31 @@ QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_contex
{ {
if (auto * join_node = global_in_or_join_node.query_node->as<JoinNode>()) if (auto * join_node = global_in_or_join_node.query_node->as<JoinNode>())
{ {
auto join_right_table_expression = join_node->getRightTableExpression(); QueryTreeNodePtr join_table_expression;
auto join_right_table_expression_node_type = join_right_table_expression->getNodeType(); const auto join_kind = join_node->getKind();
if (join_kind == JoinKind::Left || join_kind == JoinKind::Inner)
QueryTreeNodePtr subquery_node;
if (join_right_table_expression_node_type == QueryTreeNodeType::QUERY ||
join_right_table_expression_node_type == QueryTreeNodeType::UNION)
{ {
subquery_node = join_right_table_expression; join_table_expression = join_node->getRightTableExpression();
} }
else if (join_right_table_expression_node_type == QueryTreeNodeType::TABLE || else if (join_kind == JoinKind::Right)
join_right_table_expression_node_type == QueryTreeNodeType::TABLE_FUNCTION)
{ {
const auto & columns = column_source_to_columns.at(join_right_table_expression).columns; join_table_expression = join_node->getLeftTableExpression();
subquery_node = buildSubqueryToReadColumnsFromTableExpression(columns,
join_right_table_expression,
planner_context->getQueryContext());
} }
else else
{ {
throw Exception(ErrorCodes::LOGICAL_ERROR, throw Exception(
"Expected JOIN right table expression to be table, table function, query or union node. Actual {}", ErrorCodes::LOGICAL_ERROR, "Unexpected join kind: {}", join_kind);
join_right_table_expression->formatASTForErrorMessage());
} }
auto subquery_node
= getSubqueryFromTableExpression(join_table_expression, column_source_to_columns, planner_context->getQueryContext());
auto temporary_table_expression_node = executeSubqueryNode(subquery_node, auto temporary_table_expression_node = executeSubqueryNode(subquery_node,
planner_context->getMutableQueryContext(), planner_context->getMutableQueryContext(),
global_in_or_join_node.subquery_depth); global_in_or_join_node.subquery_depth);
temporary_table_expression_node->setAlias(join_right_table_expression->getAlias()); temporary_table_expression_node->setAlias(join_table_expression->getAlias());
replacement_map.emplace(join_right_table_expression.get(), std::move(temporary_table_expression_node)); replacement_map.emplace(join_table_expression.get(), std::move(temporary_table_expression_node));
continue; continue;
} }
if (auto * in_function_node = global_in_or_join_node.query_node->as<FunctionNode>()) if (auto * in_function_node = global_in_or_join_node.query_node->as<FunctionNode>())

View File

@ -35,8 +35,8 @@ def test_disk_selection(start_cluster):
node.query(query, settings=settings) node.query(query, settings=settings)
assert node.contains_in_log( assert node.contains_in_log(
"Writing part of aggregation data into temporary file /disk1/" "Writing part of aggregation data into temporary file.*/disk1/"
) )
assert node.contains_in_log( assert node.contains_in_log(
"Writing part of aggregation data into temporary file /disk2/" "Writing part of aggregation data into temporary file.*/disk2/"
) )

View File

@ -163,7 +163,6 @@ Filter column: notEquals(__table1.y, 2_UInt8)
> filter is pushed down before CreatingSets > filter is pushed down before CreatingSets
CreatingSets CreatingSets
Filter Filter
Filter
1 1
3 3
> one condition of filter is pushed down before LEFT JOIN > one condition of filter is pushed down before LEFT JOIN

View File

@ -29,6 +29,7 @@
Granules: 2/3 Granules: 2/3
----------------- -----------------
"Node Type": "ReadFromMergeTree", "Node Type": "ReadFromMergeTree",
"Node Id": "ReadFromMergeTree_0",
"Description": "default.test_index", "Description": "default.test_index",
"Indexes": [ "Indexes": [
{ {
@ -132,6 +133,7 @@
Granules: 3/6 Granules: 3/6
----------------- -----------------
"Node Type": "ReadFromMergeTree", "Node Type": "ReadFromMergeTree",
"Node Id": "ReadFromMergeTree_0",
"Description": "default.test_index", "Description": "default.test_index",
"Indexes": [ "Indexes": [
{ {

View File

@ -2,20 +2,25 @@
{ {
"Plan": { "Plan": {
"Node Type": "Union", "Node Type": "Union",
"Node Id": "Union_10",
"Plans": [ "Plans": [
{ {
"Node Type": "Expression", "Node Type": "Expression",
"Node Id": "Expression_13",
"Plans": [ "Plans": [
{ {
"Node Type": "ReadFromStorage" "Node Type": "ReadFromStorage",
"Node Id": "ReadFromStorage_0"
} }
] ]
}, },
{ {
"Node Type": "Expression", "Node Type": "Expression",
"Node Id": "Expression_16",
"Plans": [ "Plans": [
{ {
"Node Type": "ReadFromStorage" "Node Type": "ReadFromStorage",
"Node Id": "ReadFromStorage_4"
} }
] ]
} }
@ -35,6 +40,7 @@
} }
-------- --------
"Node Type": "Aggregating", "Node Type": "Aggregating",
"Node Id": "Aggregating_3",
"Header": [ "Header": [
{ {
"Name": "__table1.number", "Name": "__table1.number",
@ -73,13 +79,16 @@
], ],
-------- --------
"Node Type": "ArrayJoin", "Node Type": "ArrayJoin",
"Node Id": "ArrayJoin_4",
"Left": false, "Left": false,
"Columns": ["__table1.x", "__table1.y"], "Columns": ["__table1.x", "__table1.y"],
-------- --------
"Node Type": "Distinct", "Node Type": "Distinct",
"Node Id": "Distinct_4",
"Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"], "Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"],
-- --
"Node Type": "Distinct", "Node Type": "Distinct",
"Node Id": "Distinct_3",
"Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"], "Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"],
-------- --------
"Sort Description": [ "Sort Description": [

View File

@ -1,4 +1,4 @@
-- Tags: long, no-tsan, no-msan, no-asan, no-ubsan, no-debug, no-object-storage -- Tags: long, no-tsan, no-msan, no-asan, no-ubsan, no-debug, no-object-storage, no-random-merge-tree-settings, no-random-settings
SET max_rows_to_read = '101M'; SET max_rows_to_read = '101M';

View File

@ -77,7 +77,8 @@ SELECT
'ok', 'ok',
'fail: ' || toString(count()) || ' ' || toString(any(ProfileEvents)) 'fail: ' || toString(count()) || ' ' || toString(any(ProfileEvents))
) )
FROM system.query_log WHERE current_database = currentDatabase() FROM system.query_log
WHERE current_database = currentDatabase()
AND log_comment = '02402_external_disk_mertrics/join' AND log_comment = '02402_external_disk_mertrics/join'
AND query ILIKE 'SELECT%2097152%' AND type = 'QueryFinish'; AND query ILIKE 'SELECT%2097152%' AND type = 'QueryFinish';

View File

@ -332,13 +332,12 @@ SETTINGS optimize_aggregators_of_group_by_keys=0 -- avoid removing any() as it d
Expression (Projection) Expression (Projection)
Sorting (Sorting for ORDER BY) Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY) Expression (Before ORDER BY)
Filter ((WHERE + (Projection + Before ORDER BY))) Filter (((WHERE + (Projection + Before ORDER BY)) + HAVING))
Filter (HAVING) Aggregating
Aggregating Expression ((Before GROUP BY + Projection))
Expression ((Before GROUP BY + 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))) ReadFromSystemNumbers
ReadFromSystemNumbers
-- execute -- execute
1 1
2 2

View File

@ -28,21 +28,17 @@ WHERE type_1 = \'all\'
(Expression) (Expression)
ExpressionTransform × 2 ExpressionTransform × 2
(Filter) (Filter)
FilterTransform × 2 FilterTransform × 6
(Filter) (Aggregating)
FilterTransform × 2 ExpressionTransform × 2
(Filter) AggregatingTransform × 2
FilterTransform × 2 Copy 1 → 2
(Aggregating) (Expression)
ExpressionTransform × 2 ExpressionTransform
AggregatingTransform × 2 (Expression)
Copy 1 → 2 ExpressionTransform
(Expression) (ReadFromMergeTree)
ExpressionTransform MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression)
ExpressionTransform
(ReadFromMergeTree)
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression) (Expression)
ExpressionTransform × 2 ExpressionTransform × 2
(Filter) (Filter)
@ -68,14 +64,10 @@ ExpressionTransform × 2
ExpressionTransform × 2 ExpressionTransform × 2
AggregatingTransform × 2 AggregatingTransform × 2
Copy 1 → 2 Copy 1 → 2
(Filter) (Expression)
FilterTransform ExpressionTransform
(Filter) (ReadFromMergeTree)
FilterTransform MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression)
ExpressionTransform
(ReadFromMergeTree)
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
(Expression) (Expression)
ExpressionTransform × 2 ExpressionTransform × 2
(Aggregating) (Aggregating)

View File

@ -1,5 +1,5 @@
-- Tags: zookeeper -- Tags: zookeeper
DROP TABLE IF EXISTS join_inner_table__fuzz_146_replicated; DROP TABLE IF EXISTS join_inner_table__fuzz_146_replicated SYNC;
CREATE TABLE join_inner_table__fuzz_146_replicated CREATE TABLE join_inner_table__fuzz_146_replicated
( (
`id` UUID, `id` UUID,
@ -52,4 +52,4 @@ WHERE
GROUP BY is_initial_query, query GROUP BY is_initial_query, query
ORDER BY is_initial_query DESC, c, query; ORDER BY is_initial_query DESC, c, query;
DROP TABLE join_inner_table__fuzz_146_replicated; DROP TABLE join_inner_table__fuzz_146_replicated SYNC;

View File

@ -2,7 +2,7 @@
set parallel_replicas_prefer_local_join = 0; set parallel_replicas_prefer_local_join = 0;
-- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode.
select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -18,7 +18,7 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x
13 13 0 0 0 0 13 13 0 0 0 0
14 14 14 14 0 0 14 14 14 14 0 0
15 15 0 0 0 0 15 15 0 0 0 0
explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z;
Union Union
Expression Expression
Join Join
@ -40,8 +40,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -63,8 +62,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Union Union
@ -90,8 +88,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key select * from sub5 order by key;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
54 54 50 50 12 12 0 54 54 50 50 12 12 0
64 64 0 0 0 0 1 64 64 0 0 0 0 1
explain description=0 explain description=0
@ -100,8 +97,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key select * from sub5 order by key;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Expression Expression
@ -129,8 +125,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -152,8 +147,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Expression Expression
@ -181,8 +175,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -204,8 +197,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Expression Expression
@ -237,8 +229,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 select * from sub5;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
0 0 0 0 0 0 0 0 0 0 0 0
6 6 6 6 0 0 6 6 6 6 0 0
8 8 8 8 0 0 8 8 8 8 0 0
@ -260,30 +251,21 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 select * from sub5;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Union
Expression Expression
Join Join
Expression Expression
Join Join
Union Expression
ReadFromMemoryStorage
Expression Expression
Expression Expression
ReadFromMergeTree ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression
Expression
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression Expression
Expression ReadFromMemoryStorage
ReadFromMergeTree Expression
Expression ReadFromRemoteParallelReplicas
ReadFromRemoteParallelReplicas
-- --
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage -- RIGHT JOIN in sub5: sub5 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -291,7 +273,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; select * from sub5 order by x;
0 0 0 0 0 0 0 0 0 0 0 0
0 0 1 1 0 0 0 0 1 1 0 0
0 0 3 3 0 0 0 0 3 3 0 0
@ -313,31 +295,26 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; select * from sub5 order by x;
Expression Expression
Sorting Sorting
Expression Union
Join Expression
Union Sorting
Expression Expression
Expression Join
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Expression
Join
Union
Expression Expression
Expression ReadFromMemoryStorage
ReadFromMergeTree
Expression Expression
ReadFromRemoteParallelReplicas Join
Union Expression
Expression Expression
Expression ReadFromMergeTree
ReadFromMergeTree Expression
Expression Expression
ReadFromRemoteParallelReplicas ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
-- --
-- Subqueries for IN allowed -- Subqueries for IN allowed
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
@ -345,8 +322,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -368,8 +344,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Union Union
@ -402,7 +377,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x
SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; SETTINGS enable_parallel_replicas = 1, parallel_replicas_allow_in_with_subquery = 0;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -425,7 +400,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x
SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0;-- { echoOn } SETTINGS enable_parallel_replicas = 1, parallel_replicas_allow_in_with_subquery = 0;-- { echoOn }
Expression Expression
Sorting Sorting
Expression Expression
@ -455,7 +430,7 @@ Expression
ReadFromRemoteParallelReplicas ReadFromRemoteParallelReplicas
set parallel_replicas_prefer_local_join = 1; set parallel_replicas_prefer_local_join = 1;
-- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode.
select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -471,7 +446,7 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x
13 13 0 0 0 0 13 13 0 0 0 0
14 14 14 14 0 0 14 14 14 14 0 0
15 15 0 0 0 0 15 15 0 0 0 0
explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z;
Union Union
Expression Expression
Join Join
@ -495,8 +470,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -518,8 +492,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Union Union
@ -547,8 +520,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key select * from sub5 order by key;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
54 54 50 50 12 12 0 54 54 50 50 12 12 0
64 64 0 0 0 0 1 64 64 0 0 0 0 1
explain description=0 explain description=0
@ -557,8 +529,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key select * from sub5 order by key;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Expression Expression
@ -588,8 +559,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -611,8 +581,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Expression Expression
@ -641,8 +610,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -664,8 +632,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Expression Expression
@ -697,8 +664,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 select * from sub5;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
0 0 0 0 0 0 0 0 0 0 0 0
6 6 6 6 0 0 6 6 6 6 0 0
8 8 8 8 0 0 8 8 8 8 0 0
@ -720,30 +686,23 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 select * from sub5;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Union
Expression Expression
Join Join
Expression Expression
Join Join
Union
Expression Expression
Expression Expression
ReadFromMergeTree ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression Expression
Expression Expression
ReadFromMergeTree ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Union
Expression Expression
Expression Expression
ReadFromMergeTree ReadFromMergeTree
Expression Expression
ReadFromRemoteParallelReplicas ReadFromRemoteParallelReplicas
-- --
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage -- RIGHT JOIN in sub5: sub5 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -751,7 +710,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; select * from sub5 order by x;
0 0 0 0 0 0 0 0 0 0 0 0
0 0 1 1 0 0 0 0 1 1 0 0
0 0 3 3 0 0 0 0 3 3 0 0
@ -773,31 +732,27 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; select * from sub5 order by x;
Expression Expression
Sorting Sorting
Expression Union
Join Expression
Union Sorting
Expression Expression
Expression Join
ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
Expression
Join
Union
Expression Expression
Expression Expression
ReadFromMergeTree ReadFromMergeTree
Expression Expression
ReadFromRemoteParallelReplicas Join
Union Expression
Expression Expression
Expression ReadFromMergeTree
ReadFromMergeTree Expression
Expression Expression
ReadFromRemoteParallelReplicas ReadFromMergeTree
Expression
ReadFromRemoteParallelReplicas
-- --
-- Subqueries for IN allowed -- Subqueries for IN allowed
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
@ -805,8 +760,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -828,8 +782,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
Expression Expression
Sorting Sorting
Union Union
@ -864,7 +817,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x
SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; SETTINGS enable_parallel_replicas = 1, parallel_replicas_allow_in_with_subquery = 0;
0 0 0 0 0 0 0 0 0 0 0 0
1 1 0 0 0 0 1 1 0 0 0 0
3 3 0 0 0 0 3 3 0 0 0 0
@ -887,7 +840,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x
SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; SETTINGS enable_parallel_replicas = 1, parallel_replicas_allow_in_with_subquery = 0;
Expression Expression
Sorting Sorting
Expression Expression

View File

@ -1,16 +1,17 @@
drop table if exists tab1; drop table if exists tab1 sync;
drop table if exists tab2; drop table if exists tab2 sync;
drop table if exists tab3; drop table if exists tab3 sync;
create table tab1 (x UInt32, y UInt32, shard UInt32) engine = MergeTree order by shard; create table tab1 (x UInt32, y UInt32, shard UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02967/tab1', 'r1') order by shard;
create table tab2 (y UInt32, z UInt32) engine = MergeTree order by tuple(); create table tab2 (y UInt32, z UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02967/tab2', 'r1') order by tuple();
create table tab3 (z UInt32, a UInt32) engine = MergeTree order by tuple(); create table tab3 (z UInt32, a UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02967/tab3', 'r1') order by tuple();
insert into tab1 select number, number, number from numbers(16); insert into tab1 select number, number, number from numbers(16);
insert into tab2 select number * 2, number * 2 from numbers(8); insert into tab2 select number * 2, number * 2 from numbers(8);
insert into tab3 select number * 4, number * 4 from numbers(4); insert into tab3 select number * 4, number * 4 from numbers(4);
set parallel_replicas_local_plan=1; set enable_analyzer = 1;
set enable_parallel_replicas = 2, max_parallel_replicas = 2, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_local_plan = 1;
{% for use_global_in in [0, 1] -%} {% for use_global_in in [0, 1] -%}
@ -19,8 +20,9 @@ set parallel_replicas_local_plan=1;
set parallel_replicas_prefer_local_join = {{use_global_in}}; set parallel_replicas_prefer_local_join = {{use_global_in}};
-- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode.
select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x;
explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z;
-- --
-- The same query with cte; -- The same query with cte;
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -28,8 +30,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
explain description=0 explain description=0
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -37,8 +38,8 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
-- --
-- GROUP BY should work up to WithMergableStage -- GROUP BY should work up to WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -46,8 +47,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key select * from sub5 order by key;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
explain description=0 explain description=0
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -55,8 +55,8 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key) sub5 as (select sum(x), sum(y), sum(r.y), sum(z), sum(rr.z), sum(a), key from sub3 ll any left join sub4 rr on ll.z = rr.z group by x % 2 as key)
select * from sub5 order by key select * from sub5 order by key;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
-- --
-- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -64,8 +64,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
explain description=0 explain description=0
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -73,8 +72,8 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y order by l.x),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
-- --
-- ORDER BY in sub1 : sub1 -> WithMergableStage -- ORDER BY in sub1 : sub1 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2 order by y), with sub1 as (select x, y from tab1 where x != 2 order by y),
@ -82,8 +81,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
explain description=0 explain description=0
with sub1 as (select x, y from tab1 where x != 2 order by y), with sub1 as (select x, y from tab1 where x != 2 order by y),
@ -91,8 +89,8 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
-- --
-- RIGHT JOIN in sub3: sub3 -> WithMergableStage -- RIGHT JOIN in sub3: sub3 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -100,8 +98,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 select * from sub5;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
explain description=0 explain description=0
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -109,8 +106,8 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub2 r any right join sub1 l on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, l.y, y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 select * from sub5;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
-- --
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage -- RIGHT JOIN in sub5: sub5 -> WithMergableStage
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -118,7 +115,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; select * from sub5 order by x;
explain description=0 explain description=0
with sub1 as (select x, y from tab1 where x != 2), with sub1 as (select x, y from tab1 where x != 2),
@ -126,7 +123,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z) sub5 as (select z, a, x, y, r.y, ll.z from sub4 rr any right join sub3 ll on ll.z = rr.z)
select * from sub5 order by x SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; select * from sub5 order by x;
-- --
-- Subqueries for IN allowed -- Subqueries for IN allowed
@ -135,8 +132,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
explain description=0 explain description=0
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
@ -144,8 +140,7 @@ sub2 as (select y, z from tab2 where y != 4),
sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y), sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y = r.y),
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x;
SETTINGS enable_parallel_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1;
-- --
-- Subqueries for IN are not allowed -- Subqueries for IN are not allowed
@ -155,7 +150,7 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x
SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; SETTINGS enable_parallel_replicas = 1, parallel_replicas_allow_in_with_subquery = 0;
explain description=0 explain description=0
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
@ -164,6 +159,6 @@ sub3 as (select l.x, l.y, r.y, r.z as z from sub1 l any left join sub2 r on l.y
sub4 as (select z, a from tab3 where z != 8), sub4 as (select z, a from tab3 where z != 8),
sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z) sub5 as (select x, y, r.y, z, rr.z, a from sub3 ll any left join sub4 rr on ll.z = rr.z)
select * from sub5 order by x select * from sub5 order by x
SETTINGS enable_parallel_replicas = 1, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1, parallel_replicas_allow_in_with_subquery=0; SETTINGS enable_parallel_replicas = 1, parallel_replicas_allow_in_with_subquery = 0;
{%- endfor %} {%- endfor %}

View File

@ -2,6 +2,7 @@
SET enable_analyzer=1; SET enable_analyzer=1;
SET distributed_foreground_insert=1; SET distributed_foreground_insert=1;
DROP TABLE IF EXISTS first_table_lr SYNC;
CREATE TABLE first_table_lr CREATE TABLE first_table_lr
( (
id String, id String,
@ -11,6 +12,7 @@ ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_03080/alter', '
ORDER BY id; ORDER BY id;
DROP TABLE IF EXISTS first_table;
CREATE TABLE first_table CREATE TABLE first_table
( (
id String, id String,
@ -19,6 +21,7 @@ CREATE TABLE first_table
ENGINE = Distributed('test_shard_localhost', currentDatabase(), 'first_table_lr'); ENGINE = Distributed('test_shard_localhost', currentDatabase(), 'first_table_lr');
DROP TABLE IF EXISTS second_table_lr;
CREATE TABLE second_table_lr CREATE TABLE second_table_lr
( (
id String, id String,
@ -26,6 +29,7 @@ CREATE TABLE second_table_lr
) ENGINE = MergeTree() ) ENGINE = MergeTree()
ORDER BY id; ORDER BY id;
DROP TABLE IF EXISTS second_table;
CREATE TABLE second_table CREATE TABLE second_table
( (
id String, id String,
@ -36,6 +40,7 @@ ENGINE = Distributed('test_shard_localhost', currentDatabase(), 'second_table_lr
INSERT INTO first_table VALUES ('1', '2'), ('3', '4'); INSERT INTO first_table VALUES ('1', '2'), ('3', '4');
INSERT INTO second_table VALUES ('1', '2'), ('3', '4'); INSERT INTO second_table VALUES ('1', '2'), ('3', '4');
DROP TABLE IF EXISTS two_tables;
CREATE TABLE two_tables CREATE TABLE two_tables
( (
id String, id String,

View File

@ -6,12 +6,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
$CLICKHOUSE_CLIENT -q " $CLICKHOUSE_CLIENT -q "
DROP TABLE IF EXISTS ids;
CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple(); CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple();
INSERT INTO ids VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', 'whatever'); INSERT INTO ids VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', 'whatever');
DROP TABLE IF EXISTS data;
CREATE TABLE data (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); CREATE TABLE data (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple();
INSERT INTO data VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-01', 'CREATED'); INSERT INTO data VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-01', 'CREATED');
DROP TABLE IF EXISTS data2;
CREATE TABLE data2 (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple(); CREATE TABLE data2 (id UUID, event_time DateTime, status String) Engine=MergeTree ORDER BY tuple();
INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED'); INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED');
" "

View File

@ -0,0 +1,273 @@
inner
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
9 l9 \N 9 r9 nr9
inner subs
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
9 l9 \N 9 r9 nr9
inner expr
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
9 l9 \N 9 r9 nr9
left
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 9 r9 nr9
left subs
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 9 r9 nr9
left expr
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 9 r9 nr9
right
0 \N 6 r7 nr7
0 \N 7 r8 nr8
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
9 l9 \N 9 r9 nr9
right subs
0 \N 6 r7 nr7
0 \N 7 r8 nr8
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
9 l9 \N 9 r9 nr9
full
0 \N 6 r7 nr7
0 \N 7 r8 nr8
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 9 r9 nr9
full subs
0 \N 6 r7 nr7
0 \N 7 r8 nr8
1 l1 1 1 r1 \N
1 l1 1 1 r2 \N
2 l2 2 2 r3 \N
2 l3 3 2 r3 \N
3 l4 4 3 r4 \N
3 l4 4 3 r5 \N
4 l5 \N 4 r6 nr6
4 l6 \N 4 r6 nr6
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 9 r9 nr9
self inner
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l2 2 2 l3 3
2 l3 3 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 4 l5 \N
4 l5 \N 4 l6 \N
4 l6 \N 4 l5 \N
4 l6 \N 4 l6 \N
5 l7 \N 5 l7 \N
8 l8 \N 8 l8 \N
9 l9 \N 9 l9 \N
self inner nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
self inner nullable vs not nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l2 2
3 l4 4 2 l3 3
4 l5 \N 3 l4 4
4 l6 \N 3 l4 4
self inner nullable vs not nullable 2
4 r6 nr6 4 r6 nr6
6 r7 nr7 6 r7 nr7
7 r8 nr8 7 r8 nr8
9 r9 nr9 9 r9 nr9
self left
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l2 2 2 l3 3
2 l3 3 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 4 l5 \N
4 l5 \N 4 l6 \N
4 l6 \N 4 l5 \N
4 l6 \N 4 l6 \N
5 l7 \N 5 l7 \N
8 l8 \N 8 l8 \N
9 l9 \N 9 l9 \N
self left nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 0 \N
4 l6 \N 0 \N
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 0 \N
self left nullable vs not nullable
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l2 2
3 l4 4 2 l3 3
4 l5 \N 3 l4 4
4 l6 \N 3 l4 4
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 0 \N
self left nullable vs not nullable 2
1 r1 \N 0 \N
1 r2 \N 0 \N
2 r3 \N 0 \N
3 r4 \N 0 \N
3 r5 \N 0 \N
4 r6 nr6 4 r6 nr6
6 r7 nr7 6 r7 nr7
7 r8 nr8 7 r8 nr8
9 r9 nr9 9 r9 nr9
self right
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l2 2 2 l3 3
2 l3 3 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 4 l5 \N
4 l5 \N 4 l6 \N
4 l6 \N 4 l5 \N
4 l6 \N 4 l6 \N
5 l7 \N 5 l7 \N
8 l8 \N 8 l8 \N
9 l9 \N 9 l9 \N
self right nullable
0 \N 4 l5 \N
0 \N 4 l6 \N
0 \N 5 l7 \N
0 \N 8 l8 \N
0 \N 9 l9 \N
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
self right nullable vs not nullable
0 \N 4 l5 \N
0 \N 4 l6 \N
0 \N 5 l7 \N
0 \N 8 l8 \N
0 \N 9 l9 \N
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l2 2
3 l4 4 2 l3 3
4 l5 \N 3 l4 4
4 l6 \N 3 l4 4
self full
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l2 2 2 l3 3
2 l3 3 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 4 l5 \N
4 l5 \N 4 l6 \N
4 l6 \N 4 l5 \N
4 l6 \N 4 l6 \N
5 l7 \N 5 l7 \N
8 l8 \N 8 l8 \N
9 l9 \N 9 l9 \N
self full nullable
0 \N 4 l5 \N
0 \N 4 l6 \N
0 \N 5 l7 \N
0 \N 8 l8 \N
0 \N 9 l9 \N
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l3 3
3 l4 4 3 l4 4
4 l5 \N 0 \N
4 l6 \N 0 \N
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 0 \N
self full nullable vs not nullable
0 \N 4 l5 \N
0 \N 4 l6 \N
0 \N 5 l7 \N
0 \N 8 l8 \N
0 \N 9 l9 \N
1 l1 1 1 l1 1
2 l2 2 2 l2 2
2 l3 3 2 l2 2
3 l4 4 2 l3 3
4 l5 \N 3 l4 4
4 l6 \N 3 l4 4
5 l7 \N 0 \N
8 l8 \N 0 \N
9 l9 \N 0 \N

View File

@ -0,0 +1,73 @@
drop table if exists X sync;
drop table if exists Y sync;
set min_bytes_to_use_direct_io = 0; -- min_bytes_to_use_direct_io > 0 is broken and leads to unexpected results, https://github.com/ClickHouse/ClickHouse/issues/65690
create table X (id Int32, x_a String, x_b Nullable(Int32)) engine ReplicatedMergeTree('/clickhouse/{database}/X', '1') order by tuple();
create table Y (id Int32, y_a String, y_b Nullable(String)) engine ReplicatedMergeTree('/clickhouse/{database}/Y', '1') order by tuple();
insert into X (id, x_a, x_b) values (1, 'l1', 1), (2, 'l2', 2), (2, 'l3', 3), (3, 'l4', 4);
insert into X (id, x_a) values (4, 'l5'), (4, 'l6'), (5, 'l7'), (8, 'l8'), (9, 'l9');
insert into Y (id, y_a) values (1, 'r1'), (1, 'r2'), (2, 'r3'), (3, 'r4'), (3, 'r5');
insert into Y (id, y_a, y_b) values (4, 'r6', 'nr6'), (6, 'r7', 'nr7'), (7, 'r8', 'nr8'), (9, 'r9', 'nr9');
set enable_analyzer = 1, enable_parallel_replicas = 1, max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost';
select 'inner';
select X.*, Y.* from X inner join Y on X.id = Y.id order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'inner subs';
select s.*, j.* from (select * from X) as s inner join (select * from Y) as j on s.id = j.id order by s.id, s.x_a, s.x_b, j.id, j.y_a, j.y_b;
select 'inner expr';
select X.*, Y.* from X inner join Y on (X.id + 1) = (Y.id + 1) order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'left';
select X.*, Y.* from X left join Y on X.id = Y.id order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'left subs';
select s.*, j.* from (select * from X) as s left join (select * from Y) as j on s.id = j.id order by s.id, s.x_a, s.x_b, j.id, j.y_a, j.y_b;
select 'left expr';
select X.*, Y.* from X left join Y on (X.id + 1) = (Y.id + 1) order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'right';
select X.*, Y.* from X right join Y on X.id = Y.id order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'right subs';
select s.*, j.* from (select * from X) as s right join (select * from Y) as j on s.id = j.id order by s.id, s.x_a, s.x_b, j.id, j.y_a, j.y_b;
select 'full';
select X.*, Y.* from X full join Y on X.id = Y.id order by X.id, X.x_a, X.x_b, Y.id, Y.y_a, Y.y_b;
select 'full subs';
select s.*, j.* from (select * from X) as s full join (select * from Y) as j on s.id = j.id order by s.id, s.x_a, s.x_b, j.id, j.y_a, j.y_b;
select 'self inner';
select X.*, s.* from X inner join (select * from X) as s on X.id = s.id order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self inner nullable';
select X.*, s.* from X inner join (select * from X) as s on X.x_b = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self inner nullable vs not nullable';
select X.*, s.* from X inner join (select * from X) as s on X.id = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self inner nullable vs not nullable 2';
select Y.*, s.* from Y inner join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by Y.id, Y.y_a, Y.y_b, s.id, s.y_a, s.y_b;
select 'self left';
select X.*, s.* from X left join (select * from X) as s on X.id = s.id order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self left nullable';
select X.*, s.* from X left join (select * from X) as s on X.x_b = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self left nullable vs not nullable';
select X.*, s.* from X left join (select * from X) as s on X.id = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self left nullable vs not nullable 2';
select Y.*, s.* from Y left join (select * from Y) as s on concat('n', Y.y_a) = s.y_b order by Y.id, Y.y_a, Y.y_b, s.id, s.y_a, s.y_b;
select 'self right';
select X.*, s.* from X right join (select * from X) as s on X.id = s.id order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self right nullable';
select X.*, s.* from X right join (select * from X) as s on X.x_b = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self right nullable vs not nullable';
select X.*, s.* from X right join (select * from X) as s on X.id = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self full';
select X.*, s.* from X full join (select * from X) as s on X.id = s.id order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self full nullable';
select X.*, s.* from X full join (select * from X) as s on X.x_b = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
select 'self full nullable vs not nullable';
select X.*, s.* from X full join (select * from X) as s on X.id = s.x_b order by X.id, X.x_a, X.x_b, s.id, s.x_a, s.x_b;
drop table X sync;
drop table Y sync;

View File

@ -0,0 +1,16 @@
semi left
2 a3 2 b1
2 a6 2 b1
4 a5 4 b3
semi right
2 a3 2 b1
2 a3 2 b2
4 a5 4 b3
4 a5 4 b4
4 a5 4 b5
anti left
0 a1 0
1 a2 1
3 a4 3
anti right
0 5 b6

View File

@ -0,0 +1,26 @@
DROP TABLE IF EXISTS t1 SYNC;
DROP TABLE IF EXISTS t2 SYNC;
CREATE TABLE t1 (x UInt32, s String) engine ReplicatedMergeTree('/clickhouse/{database}/t1', '1') order by tuple();
CREATE TABLE t2 (x UInt32, s String) engine ReplicatedMergeTree('/clickhouse/{database}/t2', '1') order by tuple();
INSERT INTO t1 (x, s) VALUES (0, 'a1'), (1, 'a2'), (2, 'a3'), (3, 'a4'), (4, 'a5'), (2, 'a6');
INSERT INTO t2 (x, s) VALUES (2, 'b1'), (2, 'b2'), (4, 'b3'), (4, 'b4'), (4, 'b5'), (5, 'b6');
SET join_use_nulls = 0;
set enable_analyzer = 1, enable_parallel_replicas = 1, max_parallel_replicas = 3, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost';
SELECT 'semi left';
SELECT t1.*, t2.* FROM t1 SEMI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x, t1.s, t2.s;
SELECT 'semi right';
SELECT t1.*, t2.* FROM t1 SEMI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x, t1.s, t2.s;
SELECT 'anti left';
SELECT t1.*, t2.* FROM t1 ANTI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x, t1.s, t2.s;
SELECT 'anti right';
SELECT t1.*, t2.* FROM t1 ANTI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x, t1.s, t2.s;
DROP TABLE t1 SYNC;
DROP TABLE t2 SYNC;

View File

@ -1,2 +1,2 @@
Condition: (_CAST(toDate(ts)) in (-Inf, 1703980800]) Condition: and((materialize(auid) in [1, 1]), (_CAST(toDate(ts)) in (-Inf, 1703980800]))
Granules: 3/3 Granules: 1/3

View File

@ -0,0 +1,91 @@
[\n {\n "Plan": {\n "Node Type": "CreatingSets",\n "Node Id": "CreatingSets_22",\n "Description": "Create sets before main query execution",\n "Plans": [\n {\n "Node Type": "Expression",\n "Node Id": "Expression_18",\n "Description": "(Project names + (Before ORDER BY + Projection) [lifted up part])",\n "Plans": [\n {\n "Node Type": "Sorting",\n "Node Id": "Sorting_7",\n "Description": "Sorting for ORDER BY",\n "Plans": [\n {\n "Node Type": "Expression",\n "Node Id": "Expression_16",\n "Description": "(Before ORDER BY + Projection)",\n "Plans": [\n {\n "Node Type": "Aggregating",\n "Node Id": "Aggregating_4",\n "Plans": [\n {\n "Node Type": "Expression",\n "Node Id": "Expression_3",\n "Description": "Before GROUP BY",\n "Plans": [\n {\n "Node Type": "Filter",\n "Node Id": "Filter_14",\n "Description": "(WHERE + Change column names to column identifiers)",\n "Plans": [\n {\n "Node Type": "ReadFromMergeTree",\n "Node Id": "ReadFromMergeTree_0",\n "Description": "default.t"\n }\n ]\n }\n ]\n }\n ]\n }\n ]\n }\n ]\n }\n ]\n }\n ]\n }\n }\n]
digraph
{
rankdir="LR";
{ node [shape = rect]
n0[label="MergeTreeSelect_5"];
n1[label="FilterTransform_6"];
n2[label="ExpressionTransform_7"];
n3[label="AggregatingTransform_8"];
n4[label="Resize_9"];
n5[label="ExpressionTransform_10"];
n6[label="ExpressionTransform_11"];
n7[label="ExpressionTransform_12"];
n8[label="ExpressionTransform_13"];
n9[label="PartialSortingTransform_14"];
n10[label="PartialSortingTransform_15"];
n11[label="PartialSortingTransform_16"];
n12[label="PartialSortingTransform_17"];
n13[label="LimitsCheckingTransform_18"];
n14[label="LimitsCheckingTransform_19"];
n15[label="LimitsCheckingTransform_20"];
n16[label="LimitsCheckingTransform_21"];
n17[label="MergeSortingTransform_22"];
n18[label="MergeSortingTransform_23"];
n19[label="MergeSortingTransform_24"];
n20[label="MergeSortingTransform_25"];
n21[label="MergingSortedTransform_26"];
n22[label="ExpressionTransform_27"];
}
n0 -> n1;
n1 -> n2;
n2 -> n3;
n3 -> n4;
n4 -> n5;
n4 -> n6;
n4 -> n7;
n4 -> n8;
n5 -> n9;
n6 -> n10;
n7 -> n11;
n8 -> n12;
n9 -> n13;
n10 -> n14;
n11 -> n15;
n12 -> n16;
n13 -> n17;
n14 -> n18;
n15 -> n19;
n16 -> n20;
n17 -> n21;
n18 -> n21;
n19 -> n21;
n20 -> n21;
n21 -> n22;
}
('AggregatingTransform_8','Aggregating_4')
('ConvertingAggregatedToChunksTransform_32','')
('CreatingSetsTransform_2','CreatingSet_19')
('EmptySink_3','')
('ExpressionTransform_1','Expression_21')
('ExpressionTransform_10','Expression_16')
('ExpressionTransform_11','Expression_16')
('ExpressionTransform_12','Expression_16')
('ExpressionTransform_13','Expression_16')
('ExpressionTransform_27','Expression_18')
('ExpressionTransform_7','Expression_3')
('FilterTransform_6','Filter_14')
('LazyOutputFormat_29','')
('LimitsCheckingTransform_18','Sorting_7')
('LimitsCheckingTransform_19','Sorting_7')
('LimitsCheckingTransform_20','Sorting_7')
('LimitsCheckingTransform_21','Sorting_7')
('LimitsCheckingTransform_28','')
('MergeSortingTransform_22','Sorting_7')
('MergeSortingTransform_23','Sorting_7')
('MergeSortingTransform_24','Sorting_7')
('MergeSortingTransform_25','Sorting_7')
('MergeTreeSelect_5','ReadFromMergeTree_0')
('MergingSortedTransform_26','Sorting_7')
('NullSource_30','')
('NullSource_31','')
('NumbersRange_0','ReadFromSystemNumbers_9')
('PartialSortingTransform_14','Sorting_7')
('PartialSortingTransform_15','Sorting_7')
('PartialSortingTransform_16','Sorting_7')
('PartialSortingTransform_17','Sorting_7')
('Resize_9','Aggregating_4')

View File

@ -0,0 +1,60 @@
#!/usr/bin/env bash
# Tags: no-random-settings, no-random-merge-tree-settings
set -e
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
opts=(
--enable_analyzer=1
--max_threads=4
)
$CLICKHOUSE_CLIENT -q "
CREATE TABLE t
(
a UInt32
)
ENGINE = MergeTree
ORDER BY a;
INSERT INTO t SELECT number FROM numbers_mt(1e6);
OPTIMIZE TABLE t FINAL;
"
query="
WITH t0 AS
(
SELECT *
FROM numbers(1000)
)
SELECT a * 3
FROM t
WHERE a IN (t0)
GROUP BY a
ORDER BY a
"
$CLICKHOUSE_CLIENT "${opts[@]}" -q "EXPLAIN json=1 $query"
printf "\n\n"
$CLICKHOUSE_CLIENT "${opts[@]}" -q "SELECT replaceRegexpAll(explain, '(\w+)\(.*\)', '\\1') FROM (EXPLAIN PIPELINE compact=0,graph=1 $query)"
printf "\n\n"
query_id="03269_explain_unique_ids_$RANDOM$RANDOM"
$CLICKHOUSE_CLIENT "${opts[@]}" --log_processors_profiles=1 --query_id="$query_id" --format Null -q "$query"
$CLICKHOUSE_CLIENT -q "
SYSTEM FLUSH LOGS;
SELECT DISTINCT (replaceRegexpAll(processor_uniq_id, '(\w+)\(.*\)', '\\1'), step_uniq_id)
FROM system.processors_profile_log
WHERE query_id = '$query_id'
ORDER BY ALL;
"