mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 04:12:19 +00:00
Merge branch 'master' into vdimir/grouping_sets_aliases
This commit is contained in:
commit
e106ae1232
@ -161,6 +161,8 @@ Settings:
|
||||
- `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.
|
||||
|
||||
When `json=1` step names will contain an additional suffix with unique step identifier.
|
||||
|
||||
Example:
|
||||
|
||||
```sql
|
||||
@ -194,30 +196,25 @@ EXPLAIN json = 1, description = 0 SELECT 1 UNION ALL SELECT 2 FORMAT TSVRaw;
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Union",
|
||||
"Node Id": "Union_10",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Expression",
|
||||
"Node Id": "Expression_13",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "SettingQuotaAndLimits",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "ReadFromStorage"
|
||||
}
|
||||
]
|
||||
"Node Type": "ReadFromStorage",
|
||||
"Node Id": "ReadFromStorage_0"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Node Type": "Expression",
|
||||
"Node Id": "Expression_16",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "SettingQuotaAndLimits",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "ReadFromStorage"
|
||||
}
|
||||
]
|
||||
"Node Type": "ReadFromStorage",
|
||||
"Node Id": "ReadFromStorage_4"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -249,6 +246,7 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy;
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Expression",
|
||||
"Node Id": "Expression_5",
|
||||
"Header": [
|
||||
{
|
||||
"Name": "1",
|
||||
@ -259,18 +257,10 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy;
|
||||
"Type": "UInt16"
|
||||
}
|
||||
],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "SettingQuotaAndLimits",
|
||||
"Header": [
|
||||
{
|
||||
"Name": "dummy",
|
||||
"Type": "UInt8"
|
||||
}
|
||||
],
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "ReadFromStorage",
|
||||
"Node Id": "ReadFromStorage_0",
|
||||
"Header": [
|
||||
{
|
||||
"Name": "dummy",
|
||||
@ -280,8 +270,6 @@ EXPLAIN json = 1, description = 0, header = 1 SELECT 1, 2 + dummy;
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
]
|
||||
```
|
||||
@ -351,17 +339,31 @@ EXPLAIN json = 1, actions = 1, description = 0 SELECT 1 FORMAT TSVRaw;
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Expression",
|
||||
"Node Id": "Expression_5",
|
||||
"Expression": {
|
||||
"Inputs": [],
|
||||
"Inputs": [
|
||||
{
|
||||
"Name": "dummy",
|
||||
"Type": "UInt8"
|
||||
}
|
||||
],
|
||||
"Actions": [
|
||||
{
|
||||
"Node Type": "Column",
|
||||
"Node Type": "INPUT",
|
||||
"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)",
|
||||
"Arguments": [],
|
||||
"Removed Arguments": [],
|
||||
"Result": 0
|
||||
"Result": 1
|
||||
}
|
||||
],
|
||||
"Outputs": [
|
||||
@ -370,17 +372,12 @@ EXPLAIN json = 1, actions = 1, description = 0 SELECT 1 FORMAT TSVRaw;
|
||||
"Type": "UInt8"
|
||||
}
|
||||
],
|
||||
"Positions": [0],
|
||||
"Project Input": true
|
||||
"Positions": [1]
|
||||
},
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "SettingQuotaAndLimits",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "ReadFromStorage"
|
||||
}
|
||||
]
|
||||
"Node Type": "ReadFromStorage",
|
||||
"Node Id": "ReadFromStorage_0"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -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.
|
||||
- `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:
|
||||
|
||||
```sql
|
||||
|
@ -22,6 +22,13 @@ namespace ErrorCodes
|
||||
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>
|
||||
struct AggregationFunctionDeltaSumTimestampData
|
||||
{
|
||||
@ -37,23 +44,22 @@ template <typename ValueType, typename TimestampType>
|
||||
class AggregationFunctionDeltaSumTimestamp final
|
||||
: public IAggregateFunctionDataHelper<
|
||||
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
|
||||
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
|
||||
>
|
||||
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>>
|
||||
{
|
||||
public:
|
||||
AggregationFunctionDeltaSumTimestamp(const DataTypes & arguments, const Array & params)
|
||||
: IAggregateFunctionDataHelper<
|
||||
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
|
||||
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
|
||||
>{arguments, params, createResultType()}
|
||||
{}
|
||||
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>>{arguments, params, createResultType()}
|
||||
{
|
||||
}
|
||||
|
||||
AggregationFunctionDeltaSumTimestamp()
|
||||
: IAggregateFunctionDataHelper<
|
||||
AggregationFunctionDeltaSumTimestampData<ValueType, TimestampType>,
|
||||
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>
|
||||
>{}
|
||||
{}
|
||||
AggregationFunctionDeltaSumTimestamp<ValueType, TimestampType>>{}
|
||||
{
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
auto value = assert_cast<const ColumnVector<ValueType> &>(*columns[0]).getData()[row_num];
|
||||
auto ts = assert_cast<const ColumnVector<TimestampType> &>(*columns[1]).getData()[row_num];
|
||||
auto value = unalignedLoad<ValueType>(columns[0]->getRawData().data() + row_num * sizeof(ValueType));
|
||||
auto ts = unalignedLoad<TimestampType>(columns[1]->getRawData().data() + row_num * sizeof(TimestampType));
|
||||
|
||||
auto & data = this->data(place);
|
||||
|
||||
@ -172,10 +178,48 @@ public:
|
||||
|
||||
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(
|
||||
const String & name,
|
||||
const DataTypes & arguments,
|
||||
@ -193,8 +237,14 @@ AggregateFunctionPtr createAggregateFunctionDeltaSumTimestamp(
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}, "
|
||||
"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));
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -184,36 +184,8 @@ static IAggregateFunction * createWithDecimalType(const IDataType & argument_typ
|
||||
}
|
||||
|
||||
/** 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>
|
||||
static IAggregateFunction * createWithTwoBasicNumericTypesSecond(const IDataType & second_type, TArgs && ... args)
|
||||
{
|
||||
@ -237,46 +209,6 @@ static IAggregateFunction * createWithTwoBasicNumericTypes(const IDataType & fir
|
||||
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>
|
||||
static IAggregateFunction * createWithStringType(const IDataType & argument_type, TArgs && ... args)
|
||||
{
|
||||
|
@ -49,6 +49,7 @@
|
||||
M(TemporaryFilesForSort, "Number of temporary files created for external sorting") \
|
||||
M(TemporaryFilesForAggregation, "Number of temporary files created for external aggregation") \
|
||||
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(Read, "Number of read (read, pread, io_getevents, etc.) syscalls in fly") \
|
||||
M(RemoteRead, "Number of read with remote reader in fly") \
|
||||
|
@ -204,6 +204,16 @@ bool ThreadStatus::isQueryCanceled() const
|
||||
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()
|
||||
{
|
||||
flushUntrackedMemory();
|
||||
|
@ -11,6 +11,7 @@
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
#include <atomic>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
@ -90,6 +91,11 @@ public:
|
||||
String query_for_logs;
|
||||
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 = {};
|
||||
};
|
||||
|
||||
@ -313,6 +319,9 @@ public:
|
||||
|
||||
void initGlobalProfiler(UInt64 global_profiler_real_time_period, UInt64 global_profiler_cpu_time_period);
|
||||
|
||||
size_t getNextPlanStepIndex() const;
|
||||
size_t getNextPipelineProcessorIndex() const;
|
||||
|
||||
private:
|
||||
void applyGlobalSettings();
|
||||
void applyQuerySettings();
|
||||
|
@ -4565,7 +4565,7 @@ Possible values:
|
||||
- 0 - Disable
|
||||
- 1 - Enable
|
||||
)", 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
|
||||
)", 0) \
|
||||
DECLARE(Bool, query_plan_filter_push_down, true, R"(
|
||||
|
@ -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_handling_error", 0, 20, "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"},
|
||||
{"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"},
|
||||
|
@ -69,7 +69,7 @@ static void testCascadeBufferRedability(
|
||||
auto rbuf = wbuf_readable.tryGetReadBuffer();
|
||||
ASSERT_FALSE(!rbuf);
|
||||
|
||||
concat.appendBuffer(wrapReadBufferPointer(std::move(rbuf)));
|
||||
concat.appendBuffer(std::move(rbuf));
|
||||
}
|
||||
|
||||
std::string decoded_data;
|
||||
|
@ -335,7 +335,7 @@ Aggregator::Aggregator(const Block & header_, const Params & params_)
|
||||
: header(header_)
|
||||
, keys_positions(calculateKeysPositions(header, 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())
|
||||
{
|
||||
/// Use query-level memory tracker
|
||||
@ -1519,10 +1519,15 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, si
|
||||
Stopwatch watch;
|
||||
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);
|
||||
|
||||
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.
|
||||
|
||||
@ -1639,11 +1644,24 @@ Block Aggregator::convertOneBucketToBlock(AggregatedDataVariants & variants, Are
|
||||
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>
|
||||
void Aggregator::writeToTemporaryFileImpl(
|
||||
AggregatedDataVariants & data_variants,
|
||||
Method & method,
|
||||
TemporaryFileStream & out) const
|
||||
TemporaryBlockStreamHolder & out) const
|
||||
{
|
||||
size_t max_temporary_block_size_rows = 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)
|
||||
{
|
||||
Block block = convertOneBucketToBlock(data_variants, method, data_variants.aggregates_pool, false, bucket);
|
||||
out.write(block);
|
||||
out->write(block);
|
||||
update_max_sizes(block);
|
||||
}
|
||||
|
||||
if (params.overflow_row)
|
||||
{
|
||||
Block block = prepareBlockAndFillWithoutKey(data_variants, false, true);
|
||||
out.write(block);
|
||||
out->write(block);
|
||||
update_max_sizes(block);
|
||||
}
|
||||
|
||||
|
@ -309,9 +309,9 @@ public:
|
||||
/// For external aggregation.
|
||||
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.
|
||||
Block getHeader(bool final) const;
|
||||
@ -355,7 +355,9 @@ private:
|
||||
LoggerPtr log = getLogger("Aggregator");
|
||||
|
||||
/// 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;
|
||||
|
||||
@ -456,7 +458,7 @@ private:
|
||||
void writeToTemporaryFileImpl(
|
||||
AggregatedDataVariants & data_variants,
|
||||
Method & method,
|
||||
TemporaryFileStream & out) const;
|
||||
TemporaryBlockStreamHolder & out) const;
|
||||
|
||||
/// Merge NULL key data from hash table `src` into `dst`.
|
||||
template <typename Method, typename Table>
|
||||
|
@ -364,6 +364,8 @@ struct ContextSharedPart : boost::noncopyable
|
||||
/// Child scopes for more fine-grained accounting are created per user/query/etc.
|
||||
/// Initialized once during server startup.
|
||||
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 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.
|
||||
auto & tmp_data = root_temp_data_on_disk;
|
||||
if (tmp_data && tmp_data->getVolume())
|
||||
if (temporary_volume_legacy)
|
||||
{
|
||||
auto & disks = tmp_data->getVolume()->getDisks();
|
||||
auto & disks = temporary_volume_legacy->getDisks();
|
||||
for (auto & disk : disks)
|
||||
disk->shutdown();
|
||||
}
|
||||
@ -1184,8 +1185,8 @@ VolumePtr Context::getGlobalTemporaryVolume() const
|
||||
SharedLockGuard lock(shared->mutex);
|
||||
/// 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).
|
||||
if (shared->root_temp_data_on_disk)
|
||||
return shared->root_temp_data_on_disk->getVolume();
|
||||
if (shared->temporary_volume_legacy)
|
||||
return shared->temporary_volume_legacy;
|
||||
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.
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
fs::create_directories(path);
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -1306,7 +1311,8 @@ void Context::setTemporaryStoragePath(const String & path, size_t max_size)
|
||||
|
||||
TemporaryDataOnDiskSettings temporary_data_on_disk_settings;
|
||||
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)
|
||||
@ -1354,7 +1360,8 @@ void Context::setTemporaryStoragePolicy(const String & policy_name, size_t max_s
|
||||
|
||||
TemporaryDataOnDiskSettings temporary_data_on_disk_settings;
|
||||
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)
|
||||
@ -1378,7 +1385,8 @@ void Context::setTemporaryStorageInCache(const String & cache_disk_name, size_t
|
||||
|
||||
TemporaryDataOnDiskSettings temporary_data_on_disk_settings;
|
||||
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)
|
||||
|
@ -41,15 +41,15 @@ namespace
|
||||
class AccumulatedBlockReader
|
||||
{
|
||||
public:
|
||||
AccumulatedBlockReader(TemporaryFileStream & reader_,
|
||||
AccumulatedBlockReader(TemporaryBlockStreamReaderHolder reader_,
|
||||
std::mutex & mutex_,
|
||||
size_t result_block_size_ = 0)
|
||||
: reader(reader_)
|
||||
: reader(std::move(reader_))
|
||||
, mutex(mutex_)
|
||||
, result_block_size(result_block_size_)
|
||||
{
|
||||
if (!reader.isWriteFinished())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Reading not finished file");
|
||||
if (!reader)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Reader is nullptr");
|
||||
}
|
||||
|
||||
Block read()
|
||||
@ -63,7 +63,7 @@ namespace
|
||||
size_t rows_read = 0;
|
||||
do
|
||||
{
|
||||
Block block = reader.read();
|
||||
Block block = reader->read();
|
||||
rows_read += block.rows();
|
||||
if (!block)
|
||||
{
|
||||
@ -81,7 +81,7 @@ namespace
|
||||
}
|
||||
|
||||
private:
|
||||
TemporaryFileStream & reader;
|
||||
TemporaryBlockStreamReaderHolder reader;
|
||||
std::mutex & mutex;
|
||||
|
||||
const size_t result_block_size;
|
||||
@ -124,12 +124,12 @@ class GraceHashJoin::FileBucket : boost::noncopyable
|
||||
public:
|
||||
using BucketLock = std::unique_lock<std::mutex>;
|
||||
|
||||
explicit FileBucket(size_t bucket_index_, TemporaryFileStream & left_file_, TemporaryFileStream & right_file_, LoggerPtr log_)
|
||||
: idx{bucket_index_}
|
||||
, left_file{left_file_}
|
||||
, right_file{right_file_}
|
||||
, state{State::WRITING_BLOCKS}
|
||||
, log{log_}
|
||||
explicit FileBucket(size_t bucket_index_, TemporaryBlockStreamHolder left_file_, TemporaryBlockStreamHolder right_file_, LoggerPtr log_)
|
||||
: idx(bucket_index_)
|
||||
, left_file(std::move(left_file_))
|
||||
, right_file(std::move(right_file_))
|
||||
, state(State::WRITING_BLOCKS)
|
||||
, log(log_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -157,12 +157,6 @@ public:
|
||||
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(); }
|
||||
|
||||
AccumulatedBlockReader startJoining()
|
||||
@ -172,24 +166,21 @@ public:
|
||||
std::unique_lock<std::mutex> left_lock(left_file_mutex);
|
||||
std::unique_lock<std::mutex> right_lock(right_file_mutex);
|
||||
|
||||
left_file.finishWriting();
|
||||
right_file.finishWriting();
|
||||
|
||||
state = State::JOINING_BLOCKS;
|
||||
}
|
||||
return AccumulatedBlockReader(right_file, right_file_mutex);
|
||||
return AccumulatedBlockReader(right_file.getReadStream(), right_file_mutex);
|
||||
}
|
||||
|
||||
AccumulatedBlockReader getLeftTableReader()
|
||||
{
|
||||
ensureState(State::JOINING_BLOCKS);
|
||||
return AccumulatedBlockReader(left_file, left_file_mutex);
|
||||
return AccumulatedBlockReader(left_file.getReadStream(), left_file_mutex);
|
||||
}
|
||||
|
||||
const size_t idx;
|
||||
|
||||
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);
|
||||
|
||||
@ -199,7 +190,7 @@ private:
|
||||
if (block.rows())
|
||||
is_empty = false;
|
||||
|
||||
writer.write(block);
|
||||
writer->write(block);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -217,8 +208,8 @@ private:
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid state transition, expected {}, got {}", expected, state.load());
|
||||
}
|
||||
|
||||
TemporaryFileStream & left_file;
|
||||
TemporaryFileStream & right_file;
|
||||
TemporaryBlockStreamHolder left_file;
|
||||
TemporaryBlockStreamHolder right_file;
|
||||
mutable std::mutex left_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]}
|
||||
, left_key_names(table_join->getOnlyClause().key_names_left)
|
||||
, 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_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)
|
||||
try
|
||||
{
|
||||
auto & left_file = tmp_data->createStream(left_sample_block);
|
||||
auto & right_file = tmp_data->createStream(prepareRightBlock(right_sample_block));
|
||||
TemporaryBlockStreamHolder left_file(left_sample_block, tmp_data.get());
|
||||
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));
|
||||
}
|
||||
catch (...)
|
||||
@ -632,12 +623,9 @@ IBlocksStreamPtr GraceHashJoin::getDelayedBlocks()
|
||||
for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx)
|
||||
{
|
||||
current_bucket = buckets[bucket_idx].get();
|
||||
if (current_bucket->finished() || current_bucket->empty())
|
||||
if (current_bucket->empty())
|
||||
{
|
||||
LOG_TRACE(log, "Skipping {} {} bucket {}",
|
||||
current_bucket->finished() ? "finished" : "",
|
||||
current_bucket->empty() ? "empty" : "",
|
||||
bucket_idx);
|
||||
LOG_TRACE(log, "Skipping empty bucket {}", bucket_idx);
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -132,7 +132,7 @@ private:
|
||||
Names left_key_names;
|
||||
Names right_key_names;
|
||||
|
||||
TemporaryDataOnDiskPtr tmp_data;
|
||||
TemporaryDataOnDiskScopePtr tmp_data;
|
||||
|
||||
Buckets buckets;
|
||||
mutable SharedMutex rehash_mutex;
|
||||
|
@ -35,11 +35,6 @@
|
||||
#include <Interpreters/HashJoin/HashJoinMethods.h>
|
||||
#include <Interpreters/HashJoin/JoinUsedFlags.h>
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric TemporaryFilesForJoin;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -64,7 +59,7 @@ struct NotProcessedCrossJoin : public ExtraBlock
|
||||
{
|
||||
size_t left_position;
|
||||
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_)
|
||||
, asof_inequality(table_join->getAsofInequality())
|
||||
, data(std::make_shared<RightTableData>())
|
||||
, tmp_data(
|
||||
table_join_->getTempDataOnDisk()
|
||||
? std::make_unique<TemporaryDataOnDisk>(table_join_->getTempDataOnDisk(), CurrentMetrics::TemporaryFilesForJoin)
|
||||
: nullptr)
|
||||
, tmp_data(table_join_->getTempDataOnDisk())
|
||||
, right_sample_block(right_sample_block_)
|
||||
, max_joined_block_rows(table_join->maxJoinedBlockRows())
|
||||
, 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)
|
||||
|| (max_rows_in_join && getTotalRowCount() + block_to_save.rows() >= max_rows_in_join)))
|
||||
{
|
||||
if (tmp_stream == nullptr)
|
||||
{
|
||||
tmp_stream = &tmp_data->createStream(right_sample_block);
|
||||
}
|
||||
tmp_stream->write(block_to_save);
|
||||
if (!tmp_stream)
|
||||
tmp_stream.emplace(right_sample_block, tmp_data.get());
|
||||
|
||||
tmp_stream.value()->write(block_to_save);
|
||||
return true;
|
||||
}
|
||||
|
||||
@ -730,13 +721,14 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed)
|
||||
{
|
||||
size_t start_left_row = 0;
|
||||
size_t start_right_block = 0;
|
||||
std::unique_ptr<TemporaryFileStream::Reader> reader = nullptr;
|
||||
std::optional<TemporaryBlockStreamReaderHolder> reader;
|
||||
if (not_processed)
|
||||
{
|
||||
auto & continuation = static_cast<NotProcessedCrossJoin &>(*not_processed);
|
||||
start_left_row = continuation.left_position;
|
||||
start_right_block = continuation.right_block;
|
||||
reader = std::move(continuation.reader);
|
||||
if (continuation.reader)
|
||||
reader = std::move(*continuation.reader);
|
||||
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 (reader == nullptr)
|
||||
{
|
||||
tmp_stream->finishWritingAsyncSafe();
|
||||
if (!reader)
|
||||
reader = tmp_stream->getReadStream();
|
||||
}
|
||||
while (auto block_right = reader->read())
|
||||
|
||||
while (auto block_right = reader.value()->read())
|
||||
{
|
||||
++block_number;
|
||||
process_right_block(block_right);
|
||||
|
@ -423,8 +423,9 @@ private:
|
||||
std::vector<Sizes> key_sizes;
|
||||
|
||||
/// Needed to do external cross join
|
||||
TemporaryDataOnDiskPtr tmp_data;
|
||||
TemporaryFileStream* tmp_stream{nullptr};
|
||||
TemporaryDataOnDiskScopePtr tmp_data;
|
||||
std::optional<TemporaryBlockStreamHolder> tmp_stream;
|
||||
mutable std::once_flag finish_writing;
|
||||
|
||||
/// Block with columns from the right-side table.
|
||||
Block right_sample_block;
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <vector>
|
||||
|
||||
#include <Core/Names.h>
|
||||
#include <Core/Block.h>
|
||||
|
@ -48,6 +48,8 @@ ColumnsDescription ProcessorProfileLogElement::getColumnsDescription()
|
||||
{"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_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(output_rows);
|
||||
columns[i++]->insert(output_bytes);
|
||||
columns[i++]->insert(processor_uniq_id);
|
||||
columns[i++]->insert(step_uniq_id);
|
||||
}
|
||||
|
||||
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_description = processor->getPlanStepDescription();
|
||||
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();
|
||||
|
||||
|
@ -17,7 +17,7 @@ struct ProcessorProfileLogElement
|
||||
UInt64 id{};
|
||||
std::vector<UInt64> parent_ids;
|
||||
|
||||
UInt64 plan_step{};
|
||||
UInt64 plan_step;
|
||||
UInt64 plan_group{};
|
||||
String plan_step_name;
|
||||
String plan_step_description;
|
||||
@ -25,6 +25,8 @@ struct ProcessorProfileLogElement
|
||||
String initial_query_id;
|
||||
String query_id;
|
||||
String processor_name;
|
||||
String processor_uniq_id;
|
||||
String step_uniq_id;
|
||||
|
||||
/// Milliseconds spend in IProcessor::work()
|
||||
UInt64 elapsed_us{};
|
||||
|
@ -20,6 +20,11 @@
|
||||
#include <memory>
|
||||
#include <base/types.h>
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric TemporaryFilesForJoin;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -265,7 +270,7 @@ public:
|
||||
|
||||
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;
|
||||
|
||||
|
@ -9,13 +9,16 @@
|
||||
#include <Interpreters/Cache/FileCache.h>
|
||||
#include <Formats/NativeWriter.h>
|
||||
#include <Core/ProtocolDefines.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/SingleDiskVolume.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Disks/IO/WriteBufferFromTemporaryFile.h>
|
||||
|
||||
#include <Core/Defines.h>
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <Interpreters/Cache/WriteBufferToFileSegment.h>
|
||||
#include "Common/Exception.h"
|
||||
#include <Common/Exception.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -27,11 +30,293 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TOO_MANY_ROWS_OR_BYTES;
|
||||
extern const int INVALID_STATE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
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)
|
||||
{
|
||||
@ -54,391 +339,25 @@ void TemporaryDataOnDiskScope::deltaAllocAndCheck(ssize_t compressed_delta, ssiz
|
||||
stat.uncompressed_size += uncompressed_delta;
|
||||
}
|
||||
|
||||
TemporaryDataOnDisk::TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_)
|
||||
: TemporaryDataOnDiskScope(parent_, parent_->getSettings())
|
||||
TemporaryBlockStreamHolder::TemporaryBlockStreamHolder(const Block & header_, TemporaryDataOnDiskScope * parent_, size_t reserve_size)
|
||||
: WrapperGuard(std::make_unique<TemporaryDataBuffer>(parent_, reserve_size), DBMS_TCP_PROTOCOL_VERSION, header_)
|
||||
, header(header_)
|
||||
{}
|
||||
|
||||
TemporaryDataOnDisk::TemporaryDataOnDisk(TemporaryDataOnDiskScopePtr parent_, CurrentMetrics::Metric metric_scope)
|
||||
: TemporaryDataOnDiskScope(parent_, parent_->getSettings())
|
||||
, current_metric_scope(metric_scope)
|
||||
{}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> TemporaryDataOnDisk::createRawStream(size_t max_file_size)
|
||||
TemporaryDataBuffer::Stat TemporaryBlockStreamHolder::finishWriting() const
|
||||
{
|
||||
if (file_cache && file_cache->isInitialized())
|
||||
{
|
||||
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));
|
||||
}
|
||||
if (!holder)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary block stream is not initialized");
|
||||
|
||||
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())
|
||||
{
|
||||
auto holder = createCacheFile(max_file_size);
|
||||
|
||||
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
|
||||
}
|
||||
if (!holder)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary block stream is not initialized");
|
||||
return TemporaryBlockStreamReaderHolder(holder->read(), header, DBMS_TCP_PROTOCOL_VERSION);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -4,15 +4,21 @@
|
||||
#include <mutex>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Formats/NativeReader.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
|
||||
#include <Disks/IVolume.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
|
||||
{
|
||||
@ -25,11 +31,10 @@ namespace DB
|
||||
class TemporaryDataOnDiskScope;
|
||||
using TemporaryDataOnDiskScopePtr = std::shared_ptr<TemporaryDataOnDiskScope>;
|
||||
|
||||
class TemporaryDataOnDisk;
|
||||
using TemporaryDataOnDiskPtr = std::unique_ptr<TemporaryDataOnDisk>;
|
||||
class TemporaryDataBuffer;
|
||||
using TemporaryDataBufferPtr = std::unique_ptr<TemporaryDataBuffer>;
|
||||
|
||||
class TemporaryFileStream;
|
||||
using TemporaryFileStreamPtr = std::unique_ptr<TemporaryFileStream>;
|
||||
class TemporaryFileHolder;
|
||||
|
||||
class FileCache;
|
||||
|
||||
@ -40,15 +45,26 @@ struct TemporaryDataOnDiskSettings
|
||||
|
||||
/// Compression codec for temporary data, if empty no compression will be used. LZ4 by default
|
||||
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.
|
||||
* If limit is set, throws exception if limit is exceeded.
|
||||
* 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).
|
||||
*/
|
||||
class TemporaryDataOnDiskScope : boost::noncopyable
|
||||
class TemporaryDataOnDiskScope : boost::noncopyable, public std::enable_shared_from_this<TemporaryDataOnDiskScope>
|
||||
{
|
||||
public:
|
||||
struct StatAtomic
|
||||
@ -57,164 +73,156 @@ public:
|
||||
std::atomic<size_t> uncompressed_size;
|
||||
};
|
||||
|
||||
explicit TemporaryDataOnDiskScope(VolumePtr volume_, TemporaryDataOnDiskSettings settings_)
|
||||
: volume(std::move(volume_))
|
||||
/// Root scope
|
||||
template <typename T>
|
||||
TemporaryDataOnDiskScope(T && storage, TemporaryDataOnDiskSettings settings_)
|
||||
: file_provider(createTemporaryFileProvider(std::forward<T>(storage)))
|
||||
, 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_))
|
||||
, volume(parent->volume)
|
||||
, file_cache(parent->file_cache)
|
||||
, file_provider(parent->file_provider)
|
||||
, settings(std::move(settings_))
|
||||
{}
|
||||
|
||||
/// TODO: remove
|
||||
/// Refactor all code that uses volume directly to use TemporaryDataOnDisk.
|
||||
VolumePtr getVolume() const { return volume; }
|
||||
TemporaryDataOnDiskScopePtr childScope(CurrentMetrics::Metric current_metric);
|
||||
|
||||
const TemporaryDataOnDiskSettings & getSettings() const { return settings; }
|
||||
|
||||
protected:
|
||||
friend class TemporaryDataBuffer;
|
||||
|
||||
void deltaAllocAndCheck(ssize_t compressed_delta, ssize_t uncompressed_delta);
|
||||
|
||||
TemporaryDataOnDiskScopePtr parent = nullptr;
|
||||
|
||||
VolumePtr volume = nullptr;
|
||||
FileCache * file_cache = nullptr;
|
||||
TemporaryFileProvider file_provider;
|
||||
|
||||
StatAtomic stat;
|
||||
const TemporaryDataOnDiskSettings settings;
|
||||
};
|
||||
|
||||
/*
|
||||
* Holds the set of temporary files.
|
||||
* New file stream is created with `createStream`.
|
||||
* Streams are owned by this object and will be deleted when it is deleted.
|
||||
* It's a leaf node in temporary data scope tree.
|
||||
/** Used to hold the wrapper and wrapped object together.
|
||||
* This class provides a convenient way to manage the lifetime of both the wrapper and the wrapped object.
|
||||
* The wrapper class (Impl) stores a reference to the wrapped object (Holder), and both objects are owned by this class.
|
||||
* The lifetime of the wrapper and the wrapped object should be the same.
|
||||
* 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:
|
||||
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
|
||||
TemporaryFileStream & createStream(const Block & header, size_t max_file_size = 0);
|
||||
void reset()
|
||||
{
|
||||
impl.reset();
|
||||
holder.reset();
|
||||
}
|
||||
|
||||
/// Write raw data directly into buffer.
|
||||
/// Differences from `createStream`:
|
||||
/// 1) it doesn't account data in parent scope
|
||||
/// 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;
|
||||
protected:
|
||||
std::unique_ptr<Holder> holder;
|
||||
std::unique_ptr<Impl> impl;
|
||||
};
|
||||
|
||||
/*
|
||||
* Data can be written into this stream and then read.
|
||||
* After finish writing, call `finishWriting` and then either call `read` or 'getReadStream'(only one of the two) to read the data.
|
||||
* Account amount of data written to disk in parent scope.
|
||||
*/
|
||||
class TemporaryFileStream : boost::noncopyable
|
||||
/// Owns temporary file and provides access to it.
|
||||
/// On destruction, file is removed and all resources are freed.
|
||||
/// Lifetime of read/write buffers should be less than lifetime of TemporaryFileHolder.
|
||||
class TemporaryFileHolder
|
||||
{
|
||||
public:
|
||||
struct Reader
|
||||
{
|
||||
Reader(const String & path, const Block & header_, size_t size = 0);
|
||||
TemporaryFileHolder();
|
||||
|
||||
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;
|
||||
const size_t size;
|
||||
const std::optional<Block> header;
|
||||
virtual ~TemporaryFileHolder() = default;
|
||||
};
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> in_file_buf;
|
||||
std::unique_ptr<CompressedReadBuffer> in_compressed_buf;
|
||||
std::unique_ptr<NativeReader> in_reader;
|
||||
};
|
||||
/// Reads raw data from temporary file
|
||||
class TemporaryDataReadBuffer : public ReadBuffer
|
||||
{
|
||||
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
|
||||
{
|
||||
/// 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 uncompressed_size = 0;
|
||||
size_t num_rows = 0;
|
||||
};
|
||||
|
||||
TemporaryFileStream(TemporaryFileOnDiskHolder file_, const Block & header_, TemporaryDataOnDisk * parent_);
|
||||
TemporaryFileStream(FileSegmentsHolderPtr segments_, const Block & header_, TemporaryDataOnDisk * parent_);
|
||||
|
||||
size_t write(const Block & block);
|
||||
void flush();
|
||||
explicit TemporaryDataBuffer(TemporaryDataOnDiskScope * parent_, size_t reserve_size = 0);
|
||||
void nextImpl() override;
|
||||
void finalizeImpl() override;
|
||||
void cancelImpl() noexcept override;
|
||||
|
||||
std::unique_ptr<ReadBuffer> read();
|
||||
Stat finishWriting();
|
||||
Stat finishWritingAsyncSafe();
|
||||
bool isWriteFinished() const;
|
||||
|
||||
std::unique_ptr<Reader> getReadStream();
|
||||
String describeFilePath() const;
|
||||
|
||||
Block read();
|
||||
|
||||
String getPath() const;
|
||||
size_t getSize() const;
|
||||
|
||||
Block getHeader() const { return header; }
|
||||
|
||||
/// Read finished and file released
|
||||
bool isEof() const;
|
||||
|
||||
~TemporaryFileStream();
|
||||
~TemporaryDataBuffer() override;
|
||||
|
||||
private:
|
||||
void updateAllocAndCheck();
|
||||
|
||||
/// Release everything, close reader and writer, delete file
|
||||
void release();
|
||||
|
||||
TemporaryDataOnDisk * parent;
|
||||
|
||||
Block header;
|
||||
|
||||
/// Data can be stored in file directly or in the cache
|
||||
TemporaryFileOnDiskHolder file;
|
||||
FileSegmentsHolderPtr segment_holder;
|
||||
TemporaryDataOnDiskScope * parent;
|
||||
std::unique_ptr<TemporaryFileHolder> file_holder;
|
||||
WrapperGuard<CompressedWriteBuffer, WriteBuffer> out_compressed_buf;
|
||||
std::once_flag write_finished;
|
||||
|
||||
Stat stat;
|
||||
};
|
||||
|
||||
std::once_flag finish_writing;
|
||||
|
||||
struct OutputWriter;
|
||||
std::unique_ptr<OutputWriter> out_writer;
|
||||
/// High level interfaces for reading and writing temporary data by blocks.
|
||||
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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -934,7 +934,7 @@ static Block generateBlock(size_t size = 0)
|
||||
return block;
|
||||
}
|
||||
|
||||
static size_t readAllTemporaryData(TemporaryFileStream & stream)
|
||||
static size_t readAllTemporaryData(NativeReader & stream)
|
||||
{
|
||||
Block block;
|
||||
size_t read_rows = 0;
|
||||
@ -947,6 +947,7 @@ static size_t readAllTemporaryData(TemporaryFileStream & stream)
|
||||
}
|
||||
|
||||
TEST_F(FileCacheTest, temporaryData)
|
||||
try
|
||||
{
|
||||
ServerUUID::setRandomForUnitTests();
|
||||
DB::FileCacheSettings settings;
|
||||
@ -959,7 +960,7 @@ TEST_F(FileCacheTest, temporaryData)
|
||||
file_cache.initialize();
|
||||
|
||||
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);
|
||||
|
||||
@ -982,12 +983,17 @@ TEST_F(FileCacheTest, temporaryData)
|
||||
|
||||
size_t size_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());
|
||||
|
||||
ASSERT_GT(stream.write(generateBlock(100)), 0);
|
||||
{
|
||||
TemporaryBlockStreamHolder stream(generateBlock(), tmp_data_scope.get());
|
||||
ASSERT_GT(stream->write(generateBlock(100)), 0);
|
||||
|
||||
ASSERT_GT(file_cache.getUsedCacheSize(), 0);
|
||||
ASSERT_GT(file_cache.getFileSegmentsNum(), 0);
|
||||
@ -995,22 +1001,22 @@ TEST_F(FileCacheTest, temporaryData)
|
||||
size_t used_size_before_attempt = file_cache.getUsedCacheSize();
|
||||
/// data can't be evicted because it is still held by `some_data_holder`
|
||||
ASSERT_THROW({
|
||||
stream.write(generateBlock(2000));
|
||||
stream.flush();
|
||||
stream->write(generateBlock(2000));
|
||||
stream.finishWriting();
|
||||
}, DB::Exception);
|
||||
|
||||
ASSERT_THROW(stream.finishWriting(), DB::Exception);
|
||||
|
||||
ASSERT_EQ(file_cache.getUsedCacheSize(), used_size_before_attempt);
|
||||
}
|
||||
|
||||
{
|
||||
size_t before_used_size = file_cache.getUsedCacheSize();
|
||||
auto tmp_data = std::make_unique<TemporaryDataOnDisk>(tmp_data_scope);
|
||||
|
||||
auto write_buf_stream = tmp_data->createRawStream();
|
||||
auto write_buf_stream = std::make_unique<TemporaryDataBuffer>(tmp_data_scope.get());
|
||||
|
||||
write_buf_stream->write("1234567890", 10);
|
||||
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);
|
||||
|
||||
@ -1023,22 +1029,22 @@ TEST_F(FileCacheTest, temporaryData)
|
||||
}
|
||||
|
||||
{
|
||||
auto tmp_data = std::make_unique<TemporaryDataOnDisk>(tmp_data_scope);
|
||||
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);
|
||||
|
||||
some_data_holder.reset();
|
||||
|
||||
stream.write(generateBlock(2000));
|
||||
stream->write(generateBlock(2000));
|
||||
|
||||
auto stat = stream.finishWriting();
|
||||
stream.finishWriting();
|
||||
|
||||
ASSERT_TRUE(fs::exists(stream.getPath()));
|
||||
ASSERT_GT(fs::file_size(stream.getPath()), 100);
|
||||
String file_path = stream.getHolder()->describeFilePath().substr(strlen("fscache://"));
|
||||
|
||||
ASSERT_EQ(stat.num_rows, 2100);
|
||||
ASSERT_EQ(readAllTemporaryData(stream), 2100);
|
||||
ASSERT_TRUE(fs::exists(file_path)) << "File " << file_path << " should exist";
|
||||
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();
|
||||
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.getFileSegmentsNum(), segments_used_before_temporary_data);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
std::cerr << getCurrentExceptionMessage(true) << std::endl;
|
||||
throw;
|
||||
}
|
||||
|
||||
TEST_F(FileCacheTest, CachedReadBuffer)
|
||||
{
|
||||
@ -1148,18 +1159,22 @@ TEST_F(FileCacheTest, TemporaryDataReadBufferSize)
|
||||
DB::FileCache file_cache("cache", settings);
|
||||
file_cache.initialize();
|
||||
|
||||
auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(/*volume=*/nullptr, &file_cache, /*settings=*/TemporaryDataOnDiskSettings{});
|
||||
|
||||
auto tmp_data = std::make_unique<TemporaryDataOnDisk>(tmp_data_scope);
|
||||
auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(&file_cache, TemporaryDataOnDiskSettings{});
|
||||
|
||||
auto block = generateBlock(/*size=*/3);
|
||||
auto & stream = tmp_data->createStream(block);
|
||||
stream.write(block);
|
||||
stream.finishWriting();
|
||||
TemporaryBlockStreamHolder stream(block, tmp_data_scope.get());
|
||||
|
||||
/// 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
|
||||
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
|
||||
@ -1170,16 +1185,14 @@ TEST_F(FileCacheTest, TemporaryDataReadBufferSize)
|
||||
disk = createDisk("temporary_data_read_buffer_size_test_dir");
|
||||
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 = std::make_unique<TemporaryDataOnDisk>(tmp_data_scope);
|
||||
auto tmp_data_scope = std::make_shared<TemporaryDataOnDiskScope>(volume, TemporaryDataOnDiskSettings{});
|
||||
|
||||
auto block = generateBlock(/*size=*/3);
|
||||
auto & stream = tmp_data->createStream(block);
|
||||
stream.write(block);
|
||||
stream.finishWriting();
|
||||
TemporaryBlockStreamHolder stream(block, tmp_data_scope.get());
|
||||
stream->write(block);
|
||||
auto stat = stream.finishWriting();
|
||||
|
||||
ASSERT_EQ(stream.getSize(), 62);
|
||||
ASSERT_EQ(stat.compressed_size, 62);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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.
|
||||
/// Thus it's not safe for example to replace
|
||||
/// "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.
|
||||
if (!query.attach)
|
||||
{
|
||||
|
@ -274,7 +274,7 @@ FiltersForTableExpressionMap collectFiltersForAnalysis(const QueryTreeNodePtr &
|
||||
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)
|
||||
return {};
|
||||
|
@ -659,6 +659,7 @@ std::unique_ptr<ExpressionStep> createComputeAliasColumnsStep(
|
||||
}
|
||||
|
||||
JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression,
|
||||
const QueryTreeNodePtr & parent_join_tree,
|
||||
const SelectQueryInfo & select_query_info,
|
||||
const SelectQueryOptions & select_query_options,
|
||||
PlannerContextPtr & planner_context,
|
||||
@ -696,8 +697,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
table_expression_query_info.table_expression = table_expression;
|
||||
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.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_threads_execute_query = settings[Setting::max_threads];
|
||||
@ -912,12 +911,25 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
/// 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()
|
||||
/// 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
|
||||
= planner_context->getGlobalPlannerContext()->parallel_replicas_table
|
||||
&& !table_expression_query_info.current_table_chosen_for_reading_with_parallel_replicas;
|
||||
if (other_table_already_chosen_for_reading_with_parallel_replicas)
|
||||
planner_context->getMutableQueryContext()->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
|
||||
|
||||
const bool no_tables_or_another_table_chosen_for_reading_with_parallel_replicas_mode
|
||||
= query_context->canUseParallelReplicasOnFollower()
|
||||
&& table_node != planner_context->getGlobalPlannerContext()->parallel_replicas_table;
|
||||
if (no_tables_or_another_table_chosen_for_reading_with_parallel_replicas_mode)
|
||||
{
|
||||
auto mutable_context = Context::createCopy(query_context);
|
||||
mutable_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
|
||||
storage->read(
|
||||
query_plan,
|
||||
columns_names,
|
||||
storage_snapshot,
|
||||
table_expression_query_info,
|
||||
std::move(mutable_context),
|
||||
from_stage,
|
||||
max_block_size,
|
||||
max_streams);
|
||||
}
|
||||
else
|
||||
{
|
||||
storage->read(
|
||||
query_plan,
|
||||
columns_names,
|
||||
@ -927,6 +939,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
from_stage,
|
||||
max_block_size,
|
||||
max_streams);
|
||||
}
|
||||
|
||||
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
|
||||
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 (auto cluster = query_context->getClusterForParallelReplicas();
|
||||
@ -964,7 +990,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
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
|
||||
QueryPlan::Node * node = query_plan.getRootNode();
|
||||
@ -1794,7 +1820,8 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
|
||||
const ColumnIdentifierSet & outer_scope_columns,
|
||||
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();
|
||||
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.
|
||||
*/
|
||||
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_options,
|
||||
planner_context,
|
||||
@ -1902,7 +1931,9 @@ JoinTreeQueryPlan buildJoinTreeQueryPlan(const QueryTreeNodePtr & query_node,
|
||||
* table expression in subquery.
|
||||
*/
|
||||
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_options,
|
||||
planner_context,
|
||||
|
@ -23,6 +23,8 @@
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
#include <Storages/buildQueryTreeForShard.h>
|
||||
|
||||
#include <ranges>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace Setting
|
||||
@ -38,12 +40,12 @@ namespace ErrorCodes
|
||||
|
||||
/// Returns a list of (sub)queries (candidates) which may support parallel replicas.
|
||||
/// 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.
|
||||
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)
|
||||
{
|
||||
@ -75,7 +77,7 @@ std::stack<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTre
|
||||
{
|
||||
const auto & query_node_to_process = query_tree_node->as<QueryNode &>();
|
||||
query_tree_node = query_node_to_process.getJoinTree().get();
|
||||
res.push(&query_node_to_process);
|
||||
res.push_back(&query_node_to_process);
|
||||
break;
|
||||
}
|
||||
case QueryTreeNodeType::UNION:
|
||||
@ -98,17 +100,16 @@ std::stack<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTre
|
||||
case QueryTreeNodeType::JOIN:
|
||||
{
|
||||
const auto & join_node = query_tree_node->as<JoinNode &>();
|
||||
auto join_kind = join_node.getKind();
|
||||
auto join_strictness = join_node.getStrictness();
|
||||
const auto join_kind = join_node.getKind();
|
||||
const auto join_strictness = join_node.getStrictness();
|
||||
|
||||
bool can_parallelize_join =
|
||||
join_kind == JoinKind::Left
|
||||
|| (join_kind == JoinKind::Inner && join_strictness == JoinStrictness::All);
|
||||
|
||||
if (!can_parallelize_join)
|
||||
if (join_kind == JoinKind::Left || (join_kind == JoinKind::Inner && join_strictness == JoinStrictness::All))
|
||||
query_tree_node = join_node.getLeftTableExpression().get();
|
||||
else if (join_kind == JoinKind::Right)
|
||||
query_tree_node = join_node.getRightTableExpression().get();
|
||||
else
|
||||
return {};
|
||||
|
||||
query_tree_node = join_node.getLeftTableExpression().get();
|
||||
break;
|
||||
}
|
||||
default:
|
||||
@ -163,14 +164,27 @@ QueryTreeNodePtr replaceTablesWithDummyTables(QueryTreeNodePtr query, const Cont
|
||||
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.
|
||||
/// 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.
|
||||
const QueryNode * findQueryForParallelReplicas(
|
||||
std::stack<const QueryNode *> stack,
|
||||
std::vector<const QueryNode *> stack,
|
||||
const std::unordered_map<const QueryNode *, const QueryPlan::Node *> & mapping,
|
||||
const Settings & settings)
|
||||
{
|
||||
#ifdef DUMP_PARALLEL_REPLICAS_QUERY_CANDIDATES
|
||||
dumpStack(stack);
|
||||
#endif
|
||||
|
||||
struct Frame
|
||||
{
|
||||
const QueryPlan::Node * node = nullptr;
|
||||
@ -189,8 +203,8 @@ const QueryNode * findQueryForParallelReplicas(
|
||||
|
||||
while (!stack.empty())
|
||||
{
|
||||
const QueryNode * const subquery_node = stack.top();
|
||||
stack.pop();
|
||||
const QueryNode * const subquery_node = stack.back();
|
||||
stack.pop_back();
|
||||
|
||||
auto it = mapping.find(subquery_node);
|
||||
/// This should not happen ideally.
|
||||
@ -236,7 +250,7 @@ const QueryNode * findQueryForParallelReplicas(
|
||||
else
|
||||
{
|
||||
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.
|
||||
if (!join)
|
||||
return res;
|
||||
@ -263,7 +277,7 @@ const QueryNode * findQueryForParallelReplicas(
|
||||
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)
|
||||
return nullptr;
|
||||
@ -287,7 +301,7 @@ const QueryNode * findQueryForParallelReplicas(const QueryTreeNodePtr & query_tr
|
||||
return nullptr;
|
||||
|
||||
/// 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;
|
||||
|
||||
/// 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 * res = findQueryForParallelReplicas(new_stack, mapping, context->getSettingsRef());
|
||||
|
||||
/// Now, return a query from initial stack.
|
||||
if (res)
|
||||
{
|
||||
// find query in initial stack
|
||||
while (!new_stack.empty())
|
||||
{
|
||||
if (res == new_stack.top())
|
||||
return stack.top();
|
||||
|
||||
stack.pop();
|
||||
new_stack.pop();
|
||||
}
|
||||
if (res == new_stack.back())
|
||||
{
|
||||
res = stack.back();
|
||||
break;
|
||||
}
|
||||
|
||||
stack.pop_back();
|
||||
new_stack.pop_back();
|
||||
}
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
static const TableNode * findTableForParallelReplicas(const IQueryTreeNode * query_tree_node)
|
||||
{
|
||||
std::stack<const IQueryTreeNode *> right_join_nodes;
|
||||
while (query_tree_node || !right_join_nodes.empty())
|
||||
std::stack<const IQueryTreeNode *> join_nodes;
|
||||
while (query_tree_node || !join_nodes.empty())
|
||||
{
|
||||
if (!query_tree_node)
|
||||
{
|
||||
query_tree_node = right_join_nodes.top();
|
||||
right_join_nodes.pop();
|
||||
query_tree_node = join_nodes.top();
|
||||
join_nodes.pop();
|
||||
}
|
||||
|
||||
auto join_tree_node_type = query_tree_node->getNodeType();
|
||||
@ -383,8 +399,23 @@ static const TableNode * findTableForParallelReplicas(const IQueryTreeNode * que
|
||||
case QueryTreeNodeType::JOIN:
|
||||
{
|
||||
const auto & join_node = query_tree_node->as<JoinNode &>();
|
||||
const auto join_kind = join_node.getKind();
|
||||
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();
|
||||
right_join_nodes.push(join_node.getRightTableExpression().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;
|
||||
}
|
||||
default:
|
||||
@ -400,7 +431,7 @@ static const TableNode * findTableForParallelReplicas(const IQueryTreeNode * que
|
||||
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)
|
||||
return nullptr;
|
||||
|
@ -15,10 +15,10 @@ struct SelectQueryOptions;
|
||||
|
||||
/// Find a query which can be executed with parallel replicas up to WithMergableStage.
|
||||
/// 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.
|
||||
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;
|
||||
|
||||
|
@ -79,7 +79,7 @@ bool ExecutionThreadContext::executeTask()
|
||||
|
||||
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);
|
||||
}
|
||||
std::optional<Stopwatch> execution_time_watch;
|
||||
|
@ -10,6 +10,20 @@
|
||||
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)
|
||||
{
|
||||
query_plan_step = step;
|
||||
@ -18,6 +32,7 @@ void IProcessor::setQueryPlanStep(IQueryPlanStep * step, size_t group)
|
||||
{
|
||||
plan_step_name = step->getName();
|
||||
plan_step_description = step->getStepDescription();
|
||||
step_uniq_id = step->getUniqID();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,9 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Processors/Port.h>
|
||||
#include <Processors/QueryPlan/IQueryPlanStep.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
|
||||
#include <memory>
|
||||
|
||||
class EventCounter;
|
||||
|
||||
@ -121,19 +124,14 @@ protected:
|
||||
OutputPorts outputs;
|
||||
|
||||
public:
|
||||
IProcessor() = default;
|
||||
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;
|
||||
}
|
||||
IProcessor(InputPorts inputs_, OutputPorts outputs_);
|
||||
|
||||
virtual String getName() const = 0;
|
||||
|
||||
String getUniqID() const { return fmt::format("{}_{}", getName(), processor_index); }
|
||||
|
||||
enum class Status : uint8_t
|
||||
{
|
||||
/// Processor needs some data at its inputs to proceed.
|
||||
@ -314,6 +312,7 @@ public:
|
||||
void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0);
|
||||
|
||||
IQueryPlanStep * getQueryPlanStep() const { return query_plan_step; }
|
||||
const String & getStepUniqID() const { return step_uniq_id; }
|
||||
size_t getQueryPlanStepGroup() const { return query_plan_step_group; }
|
||||
const String & getPlanStepName() const { return plan_step_name; }
|
||||
const String & getPlanStepDescription() const { return plan_step_description; }
|
||||
@ -407,7 +406,10 @@ private:
|
||||
size_t stream_number = NO_STREAM;
|
||||
|
||||
IQueryPlanStep * query_plan_step = nullptr;
|
||||
String step_uniq_id;
|
||||
size_t query_plan_step_group = 0;
|
||||
|
||||
size_t processor_index = 0;
|
||||
String plan_step_name;
|
||||
String plan_step_description;
|
||||
};
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/JSONBuilder.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypesNumber.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);
|
||||
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())
|
||||
cast_type = std::make_shared<DataTypeNullable>(std::move(cast_type));
|
||||
|
||||
|
@ -10,6 +10,11 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
IQueryPlanStep::IQueryPlanStep()
|
||||
{
|
||||
step_index = CurrentThread::isInitialized() ? CurrentThread::get().getNextPlanStepIndex() : 0;
|
||||
}
|
||||
|
||||
void IQueryPlanStep::updateInputHeaders(Headers input_headers_)
|
||||
{
|
||||
input_headers = std::move(input_headers_);
|
||||
|
@ -1,8 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Core/SortDescription.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
|
||||
|
||||
#include <fmt/core.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -26,6 +31,8 @@ using Headers = std::vector<Header>;
|
||||
class IQueryPlanStep
|
||||
{
|
||||
public:
|
||||
IQueryPlanStep();
|
||||
|
||||
virtual ~IQueryPlanStep() = default;
|
||||
|
||||
virtual String getName() const = 0;
|
||||
@ -77,6 +84,8 @@ public:
|
||||
|
||||
/// 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
|
||||
String getUniqID() const { return fmt::format("{}_{}", getName(), step_index); }
|
||||
|
||||
/// (e.g. you correctly remove / add columns).
|
||||
void updateInputHeaders(Headers input_headers_);
|
||||
void updateInputHeader(Header input_header, size_t idx = 0);
|
||||
@ -95,6 +104,9 @@ protected:
|
||||
Processors processors;
|
||||
|
||||
static void describePipeline(const Processors & processors, FormatSettings & settings);
|
||||
|
||||
private:
|
||||
size_t step_index = 0;
|
||||
};
|
||||
|
||||
using QueryPlanStepPtr = std::unique_ptr<IQueryPlanStep>;
|
||||
|
@ -3,12 +3,15 @@
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/IJoin.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <Interpreters/TableJoin.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Processors/QueryPlan/ConvertingActions.h>
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
#include <Processors/QueryPlan/ISourceStep.h>
|
||||
#include <Processors/QueryPlan/JoinStep.h>
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
@ -62,7 +65,14 @@ std::pair<std::unique_ptr<QueryPlan>, bool> createLocalPlanForParallelReplicas(
|
||||
break;
|
||||
|
||||
if (!node->children.empty())
|
||||
{
|
||||
// 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
|
||||
node = nullptr;
|
||||
}
|
||||
|
@ -207,6 +207,7 @@ QueryPipelineBuilderPtr QueryPlan::buildQueryPipeline(
|
||||
static void explainStep(const IQueryPlanStep & step, JSONBuilder::JSONMap & map, const QueryPlan::ExplainPlanOptions & options)
|
||||
{
|
||||
map.add("Node Type", step.getName());
|
||||
map.add("Node Id", step.getUniqID());
|
||||
|
||||
if (options.description)
|
||||
{
|
||||
|
@ -282,9 +282,9 @@ void SortingStep::mergeSorting(
|
||||
if (increase_sort_description_compile_attempts)
|
||||
increase_sort_description_compile_attempts = false;
|
||||
|
||||
auto tmp_data_on_disk = sort_settings.tmp_data
|
||||
? std::make_unique<TemporaryDataOnDisk>(sort_settings.tmp_data, CurrentMetrics::TemporaryFilesForSort)
|
||||
: std::unique_ptr<TemporaryDataOnDisk>();
|
||||
TemporaryDataOnDiskScopePtr tmp_data_on_disk = nullptr;
|
||||
if (sort_settings.tmp_data)
|
||||
tmp_data_on_disk = sort_settings.tmp_data->childScope(CurrentMetrics::TemporaryFilesForSort);
|
||||
|
||||
return std::make_shared<MergeSortingTransform>(
|
||||
header,
|
||||
|
@ -54,9 +54,9 @@ namespace
|
||||
class SourceFromNativeStream : public ISource
|
||||
{
|
||||
public:
|
||||
explicit SourceFromNativeStream(TemporaryFileStream * tmp_stream_)
|
||||
: ISource(tmp_stream_->getHeader())
|
||||
, tmp_stream(tmp_stream_)
|
||||
explicit SourceFromNativeStream(const Block & header, TemporaryBlockStreamReaderHolder tmp_stream_)
|
||||
: ISource(header)
|
||||
, tmp_stream(std::move(tmp_stream_))
|
||||
{}
|
||||
|
||||
String getName() const override { return "SourceFromNativeStream"; }
|
||||
@ -69,7 +69,7 @@ namespace
|
||||
auto block = tmp_stream->read();
|
||||
if (!block)
|
||||
{
|
||||
tmp_stream = nullptr;
|
||||
tmp_stream.reset();
|
||||
return {};
|
||||
}
|
||||
return convertToChunk(block);
|
||||
@ -78,7 +78,7 @@ namespace
|
||||
std::optional<ReadProgress> getReadProgress() override { return std::nullopt; }
|
||||
|
||||
private:
|
||||
TemporaryFileStream * tmp_stream;
|
||||
TemporaryBlockStreamReaderHolder tmp_stream;
|
||||
};
|
||||
}
|
||||
|
||||
@ -811,15 +811,18 @@ void AggregatingTransform::initGenerate()
|
||||
|
||||
Pipes pipes;
|
||||
/// 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();
|
||||
for (auto * tmp_stream : tmp_data.getStreams())
|
||||
pipes.emplace_back(Pipe(std::make_unique<SourceFromNativeStream>(tmp_stream)));
|
||||
tmp_files = aggregator.detachTemporaryData();
|
||||
num_streams += tmp_files.size();
|
||||
|
||||
num_streams += tmp_data.getStreams().size();
|
||||
compressed_size += tmp_data.getStat().compressed_size;
|
||||
uncompressed_size += tmp_data.getStat().uncompressed_size;
|
||||
for (auto & tmp_stream : tmp_files)
|
||||
{
|
||||
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(
|
||||
|
@ -216,6 +216,8 @@ private:
|
||||
|
||||
RowsBeforeStepCounterPtr rows_before_aggregation;
|
||||
|
||||
std::list<TemporaryBlockStreamHolder> tmp_files;
|
||||
|
||||
void initGenerate();
|
||||
};
|
||||
|
||||
|
@ -27,15 +27,20 @@ namespace ProfileEvents
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
class BufferingToFileTransform : public IAccumulatingTransform
|
||||
{
|
||||
public:
|
||||
BufferingToFileTransform(const Block & header, TemporaryFileStream & tmp_stream_, LoggerPtr log_)
|
||||
BufferingToFileTransform(const Block & header, TemporaryBlockStreamHolder tmp_stream_, LoggerPtr log_)
|
||||
: IAccumulatingTransform(header, header)
|
||||
, tmp_stream(tmp_stream_)
|
||||
, tmp_stream(std::move(tmp_stream_))
|
||||
, 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);
|
||||
}
|
||||
|
||||
@ -44,14 +49,15 @@ public:
|
||||
void consume(Chunk chunk) override
|
||||
{
|
||||
Block block = getInputPort().getHeader().cloneWithColumns(chunk.detachColumns());
|
||||
tmp_stream.write(block);
|
||||
tmp_stream->write(block);
|
||||
}
|
||||
|
||||
Chunk generate() override
|
||||
{
|
||||
if (!tmp_stream.isWriteFinished())
|
||||
if (!tmp_read_stream)
|
||||
{
|
||||
auto stat = tmp_stream.finishWriting();
|
||||
tmp_read_stream = tmp_stream.getReadStream();
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::ExternalProcessingCompressedBytesTotal, stat.compressed_size);
|
||||
ProfileEvents::increment(ProfileEvents::ExternalProcessingUncompressedBytesTotal, stat.uncompressed_size);
|
||||
@ -59,10 +65,11 @@ public:
|
||||
ProfileEvents::increment(ProfileEvents::ExternalSortUncompressedBytes, stat.uncompressed_size);
|
||||
|
||||
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)
|
||||
return {};
|
||||
|
||||
@ -71,7 +78,8 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
TemporaryFileStream & tmp_stream;
|
||||
TemporaryBlockStreamHolder tmp_stream;
|
||||
std::optional<TemporaryBlockStreamReaderHolder> tmp_read_stream;
|
||||
|
||||
LoggerPtr log;
|
||||
};
|
||||
@ -86,7 +94,7 @@ MergeSortingTransform::MergeSortingTransform(
|
||||
size_t max_bytes_before_remerge_,
|
||||
double remerge_lowered_memory_bytes_ratio_,
|
||||
size_t max_bytes_before_external_sort_,
|
||||
TemporaryDataOnDiskPtr tmp_data_,
|
||||
TemporaryDataOnDiskScopePtr tmp_data_,
|
||||
size_t min_free_disk_space_)
|
||||
: SortingTransform(header, description_, max_merged_block_size_, limit_, increase_sort_description_compile_attempts)
|
||||
, 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 (!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
|
||||
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;
|
||||
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);
|
||||
}
|
||||
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);
|
||||
|
||||
@ -223,14 +235,14 @@ void MergeSortingTransform::generate()
|
||||
{
|
||||
if (!generated_prefix)
|
||||
{
|
||||
size_t num_tmp_files = tmp_data ? tmp_data->getStreams().size() : 0;
|
||||
if (num_tmp_files == 0)
|
||||
merge_sorter
|
||||
= std::make_unique<MergeSorter>(header_without_constants, std::move(chunks), description, max_merged_block_size, limit);
|
||||
if (temporary_files_num == 0)
|
||||
{
|
||||
merge_sorter = std::make_unique<MergeSorter>(header_without_constants, std::move(chunks), description, max_merged_block_size, limit);
|
||||
}
|
||||
else
|
||||
{
|
||||
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>(
|
||||
header_without_constants, std::move(chunks), description, max_merged_block_size, limit));
|
||||
|
@ -29,7 +29,7 @@ public:
|
||||
size_t max_bytes_before_remerge_,
|
||||
double remerge_lowered_memory_bytes_ratio_,
|
||||
size_t max_bytes_before_external_sort_,
|
||||
TemporaryDataOnDiskPtr tmp_data_,
|
||||
TemporaryDataOnDiskScopePtr tmp_data_,
|
||||
size_t min_free_disk_space_);
|
||||
|
||||
String getName() const override { return "MergeSortingTransform"; }
|
||||
@ -45,7 +45,8 @@ private:
|
||||
size_t max_bytes_before_remerge;
|
||||
double remerge_lowered_memory_bytes_ratio;
|
||||
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 max_block_bytes;
|
||||
|
||||
|
@ -398,10 +398,10 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
|
||||
|
||||
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.
|
||||
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.
|
||||
bool default_totals = false;
|
||||
|
@ -30,7 +30,7 @@ void printPipeline(const Processors & processors, const Statuses & statuses, Wri
|
||||
for (const auto & processor : processors)
|
||||
{
|
||||
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())
|
||||
{
|
||||
|
@ -170,15 +170,16 @@ void HTTPHandler::pushDelayedResults(Output & used_output)
|
||||
|
||||
for (auto & write_buf : write_buffers)
|
||||
{
|
||||
if (!write_buf)
|
||||
continue;
|
||||
|
||||
IReadableWriteBuffer * write_buf_concrete = dynamic_cast<IReadableWriteBuffer *>(write_buf.get());
|
||||
if (write_buf_concrete)
|
||||
if (auto * write_buf_concrete = dynamic_cast<TemporaryDataBuffer *>(write_buf.get()))
|
||||
{
|
||||
ReadBufferPtr reread_buf = write_buf_concrete->tryGetReadBuffer();
|
||||
if (reread_buf)
|
||||
read_buffers.emplace_back(wrapReadBufferPointer(reread_buf));
|
||||
if (auto reread_buf = write_buf_concrete->read())
|
||||
read_buffers.emplace_back(std::move(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)
|
||||
{
|
||||
CascadeWriteBuffer::WriteBufferPtrs cascade_buffer1;
|
||||
CascadeWriteBuffer::WriteBufferConstructors cascade_buffer2;
|
||||
CascadeWriteBuffer::WriteBufferPtrs cascade_buffers;
|
||||
CascadeWriteBuffer::WriteBufferConstructors cascade_buffers_lazy;
|
||||
|
||||
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)
|
||||
{
|
||||
auto tmp_data = std::make_shared<TemporaryDataOnDisk>(server.context()->getTempDataOnDisk());
|
||||
|
||||
auto create_tmp_disk_buffer = [tmp_data] (const WriteBufferPtr &) -> WriteBufferPtr {
|
||||
return tmp_data->createRawStream();
|
||||
};
|
||||
|
||||
cascade_buffer2.emplace_back(std::move(create_tmp_disk_buffer));
|
||||
auto tmp_data = server.context()->getTempDataOnDisk();
|
||||
cascade_buffers_lazy.emplace_back([tmp_data](const WriteBufferPtr &) -> WriteBufferPtr
|
||||
{
|
||||
return std::make_unique<TemporaryDataBuffer>(tmp_data.get());
|
||||
});
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -351,10 +350,10 @@ void HTTPHandler::processQuery(
|
||||
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();
|
||||
}
|
||||
else
|
||||
|
@ -65,6 +65,11 @@ namespace ProfileEvents
|
||||
extern const Event MergeProjectionStageExecuteMilliseconds;
|
||||
}
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric TemporaryFilesForMerge;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace Setting
|
||||
@ -124,6 +129,7 @@ static ColumnsStatistics getStatisticsForColumns(
|
||||
return all_statistics;
|
||||
}
|
||||
|
||||
|
||||
/// Manages the "rows_sources" temporary file that is used during vertical merge.
|
||||
class RowsSourcesTemporaryFile : public ITemporaryFileLookup
|
||||
{
|
||||
@ -132,9 +138,7 @@ public:
|
||||
static constexpr auto FILE_ID = "rows_sources";
|
||||
|
||||
explicit RowsSourcesTemporaryFile(TemporaryDataOnDiskScopePtr temporary_data_on_disk_)
|
||||
: tmp_disk(std::make_unique<TemporaryDataOnDisk>(temporary_data_on_disk_))
|
||||
, uncompressed_write_buffer(tmp_disk->createRawStream())
|
||||
, tmp_file_name_on_disk(uncompressed_write_buffer->getFileName())
|
||||
: temporary_data_on_disk(temporary_data_on_disk_->childScope(CurrentMetrics::TemporaryFilesForMerge))
|
||||
{
|
||||
}
|
||||
|
||||
@ -143,11 +147,11 @@ public:
|
||||
if (name != FILE_ID)
|
||||
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");
|
||||
|
||||
write_buffer = (std::make_unique<CompressedWriteBuffer>(*uncompressed_write_buffer));
|
||||
return *write_buffer;
|
||||
tmp_data_buffer = std::make_unique<TemporaryDataBuffer>(temporary_data_on_disk.get());
|
||||
return *tmp_data_buffer;
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBuffer> getTemporaryFileForReading(const String & name) override
|
||||
@ -163,25 +167,24 @@ public:
|
||||
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.
|
||||
auto raw_file_read_buffer = std::make_unique<ReadBufferFromFile>(tmp_file_name_on_disk);
|
||||
return std::make_unique<CompressedReadBufferFromFile>(std::move(raw_file_read_buffer));
|
||||
return tmp_data_buffer->read();
|
||||
}
|
||||
|
||||
/// Returns written data size in bytes
|
||||
size_t finalizeWriting()
|
||||
{
|
||||
write_buffer->finalize();
|
||||
uncompressed_write_buffer->finalize();
|
||||
if (!tmp_data_buffer)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Temporary file was not requested for writing");
|
||||
|
||||
auto stat = tmp_data_buffer->finishWriting();
|
||||
finalized = true;
|
||||
final_size = write_buffer->count();
|
||||
final_size = stat.uncompressed_size;
|
||||
return final_size;
|
||||
}
|
||||
|
||||
private:
|
||||
std::unique_ptr<TemporaryDataOnDisk> tmp_disk;
|
||||
std::unique_ptr<WriteBufferFromFileBase> uncompressed_write_buffer;
|
||||
std::unique_ptr<WriteBuffer> write_buffer;
|
||||
const String tmp_file_name_on_disk;
|
||||
std::unique_ptr<TemporaryDataBuffer> tmp_data_buffer;
|
||||
TemporaryDataOnDiskScopePtr temporary_data_on_disk;
|
||||
bool finalized = false;
|
||||
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
|
||||
/// skipped writing rows_sources file. Otherwise rows_sources_count must be equal to the total
|
||||
/// 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)
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
@ -881,6 +885,7 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const
|
||||
"of bytes written to rows_sources file ({}). It is a bug.",
|
||||
sum_input_rows_exact, input_rows_filtered, rows_sources_count);
|
||||
|
||||
|
||||
ctx->it_name_and_type = global_ctx->gathering_columns.cbegin();
|
||||
|
||||
const auto & settings = global_ctx->context->getSettingsRef();
|
||||
|
@ -162,8 +162,6 @@ struct SelectQueryInfo
|
||||
/// It's guaranteed to be present in JOIN TREE of `query_tree`
|
||||
QueryTreeNodePtr table_expression;
|
||||
|
||||
bool current_table_chosen_for_reading_with_parallel_replicas = false;
|
||||
|
||||
/// Table expression modifiers for storage
|
||||
std::optional<TableExpressionModifiers> table_expression_modifiers;
|
||||
|
||||
|
@ -276,9 +276,7 @@ void StorageMergeTree::read(
|
||||
}
|
||||
|
||||
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower()
|
||||
&& 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);
|
||||
&& local_context->getSettingsRef()[Setting::parallel_replicas_for_non_replicated_merge_tree];
|
||||
|
||||
if (auto plan = reader.read(
|
||||
column_names,
|
||||
|
@ -5640,10 +5640,7 @@ void StorageReplicatedMergeTree::readLocalImpl(
|
||||
const size_t max_block_size,
|
||||
const size_t num_streams)
|
||||
{
|
||||
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);
|
||||
|
||||
const bool enable_parallel_reading = local_context->canUseParallelReplicasOnFollower();
|
||||
auto plan = reader.read(
|
||||
column_names, storage_snapshot, query_info,
|
||||
local_context, max_block_size, num_streams,
|
||||
|
@ -314,6 +314,35 @@ TableNodePtr executeSubqueryNode(const QueryTreeNodePtr & subquery_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)
|
||||
@ -335,37 +364,31 @@ QueryTreeNodePtr buildQueryTreeForShard(const PlannerContextPtr & planner_contex
|
||||
{
|
||||
if (auto * join_node = global_in_or_join_node.query_node->as<JoinNode>())
|
||||
{
|
||||
auto join_right_table_expression = join_node->getRightTableExpression();
|
||||
auto join_right_table_expression_node_type = join_right_table_expression->getNodeType();
|
||||
|
||||
QueryTreeNodePtr subquery_node;
|
||||
|
||||
if (join_right_table_expression_node_type == QueryTreeNodeType::QUERY ||
|
||||
join_right_table_expression_node_type == QueryTreeNodeType::UNION)
|
||||
QueryTreeNodePtr join_table_expression;
|
||||
const auto join_kind = join_node->getKind();
|
||||
if (join_kind == JoinKind::Left || join_kind == JoinKind::Inner)
|
||||
{
|
||||
subquery_node = join_right_table_expression;
|
||||
join_table_expression = join_node->getRightTableExpression();
|
||||
}
|
||||
else if (join_right_table_expression_node_type == QueryTreeNodeType::TABLE ||
|
||||
join_right_table_expression_node_type == QueryTreeNodeType::TABLE_FUNCTION)
|
||||
else if (join_kind == JoinKind::Right)
|
||||
{
|
||||
const auto & columns = column_source_to_columns.at(join_right_table_expression).columns;
|
||||
subquery_node = buildSubqueryToReadColumnsFromTableExpression(columns,
|
||||
join_right_table_expression,
|
||||
planner_context->getQueryContext());
|
||||
join_table_expression = join_node->getLeftTableExpression();
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Expected JOIN right table expression to be table, table function, query or union node. Actual {}",
|
||||
join_right_table_expression->formatASTForErrorMessage());
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR, "Unexpected join kind: {}", join_kind);
|
||||
}
|
||||
|
||||
auto subquery_node
|
||||
= getSubqueryFromTableExpression(join_table_expression, column_source_to_columns, planner_context->getQueryContext());
|
||||
|
||||
auto temporary_table_expression_node = executeSubqueryNode(subquery_node,
|
||||
planner_context->getMutableQueryContext(),
|
||||
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;
|
||||
}
|
||||
if (auto * in_function_node = global_in_or_join_node.query_node->as<FunctionNode>())
|
||||
|
@ -35,8 +35,8 @@ def test_disk_selection(start_cluster):
|
||||
|
||||
node.query(query, settings=settings)
|
||||
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(
|
||||
"Writing part of aggregation data into temporary file /disk2/"
|
||||
"Writing part of aggregation data into temporary file.*/disk2/"
|
||||
)
|
||||
|
@ -163,7 +163,6 @@ Filter column: notEquals(__table1.y, 2_UInt8)
|
||||
> filter is pushed down before CreatingSets
|
||||
CreatingSets
|
||||
Filter
|
||||
Filter
|
||||
1
|
||||
3
|
||||
> one condition of filter is pushed down before LEFT JOIN
|
||||
|
@ -29,6 +29,7 @@
|
||||
Granules: 2/3
|
||||
-----------------
|
||||
"Node Type": "ReadFromMergeTree",
|
||||
"Node Id": "ReadFromMergeTree_0",
|
||||
"Description": "default.test_index",
|
||||
"Indexes": [
|
||||
{
|
||||
@ -132,6 +133,7 @@
|
||||
Granules: 3/6
|
||||
-----------------
|
||||
"Node Type": "ReadFromMergeTree",
|
||||
"Node Id": "ReadFromMergeTree_0",
|
||||
"Description": "default.test_index",
|
||||
"Indexes": [
|
||||
{
|
||||
|
@ -2,20 +2,25 @@
|
||||
{
|
||||
"Plan": {
|
||||
"Node Type": "Union",
|
||||
"Node Id": "Union_10",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "Expression",
|
||||
"Node Id": "Expression_13",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "ReadFromStorage"
|
||||
"Node Type": "ReadFromStorage",
|
||||
"Node Id": "ReadFromStorage_0"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Node Type": "Expression",
|
||||
"Node Id": "Expression_16",
|
||||
"Plans": [
|
||||
{
|
||||
"Node Type": "ReadFromStorage"
|
||||
"Node Type": "ReadFromStorage",
|
||||
"Node Id": "ReadFromStorage_4"
|
||||
}
|
||||
]
|
||||
}
|
||||
@ -35,6 +40,7 @@
|
||||
}
|
||||
--------
|
||||
"Node Type": "Aggregating",
|
||||
"Node Id": "Aggregating_3",
|
||||
"Header": [
|
||||
{
|
||||
"Name": "__table1.number",
|
||||
@ -73,13 +79,16 @@
|
||||
],
|
||||
--------
|
||||
"Node Type": "ArrayJoin",
|
||||
"Node Id": "ArrayJoin_4",
|
||||
"Left": false,
|
||||
"Columns": ["__table1.x", "__table1.y"],
|
||||
--------
|
||||
"Node Type": "Distinct",
|
||||
"Node Id": "Distinct_4",
|
||||
"Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"],
|
||||
--
|
||||
"Node Type": "Distinct",
|
||||
"Node Id": "Distinct_3",
|
||||
"Columns": ["intDiv(__table1.number, 2_UInt8)", "intDiv(__table1.number, 3_UInt8)"],
|
||||
--------
|
||||
"Sort Description": [
|
||||
|
@ -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';
|
||||
|
||||
|
@ -77,7 +77,8 @@ SELECT
|
||||
'ok',
|
||||
'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 query ILIKE 'SELECT%2097152%' AND type = 'QueryFinish';
|
||||
|
||||
|
@ -332,8 +332,7 @@ SETTINGS optimize_aggregators_of_group_by_keys=0 -- avoid removing any() as it d
|
||||
Expression (Projection)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression (Before ORDER BY)
|
||||
Filter ((WHERE + (Projection + Before ORDER BY)))
|
||||
Filter (HAVING)
|
||||
Filter (((WHERE + (Projection + Before ORDER BY)) + HAVING))
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + Projection))
|
||||
Sorting (Sorting for ORDER BY)
|
||||
|
@ -28,11 +28,7 @@ WHERE type_1 = \'all\'
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(Filter)
|
||||
FilterTransform × 2
|
||||
(Filter)
|
||||
FilterTransform × 2
|
||||
(Filter)
|
||||
FilterTransform × 2
|
||||
FilterTransform × 6
|
||||
(Aggregating)
|
||||
ExpressionTransform × 2
|
||||
AggregatingTransform × 2
|
||||
@ -68,10 +64,6 @@ ExpressionTransform × 2
|
||||
ExpressionTransform × 2
|
||||
AggregatingTransform × 2
|
||||
Copy 1 → 2
|
||||
(Filter)
|
||||
FilterTransform
|
||||
(Filter)
|
||||
FilterTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
|
@ -1,5 +1,5 @@
|
||||
-- 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
|
||||
(
|
||||
`id` UUID,
|
||||
@ -52,4 +52,4 @@ WHERE
|
||||
GROUP BY is_initial_query, 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;
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
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.
|
||||
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
|
||||
1 1 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
|
||||
14 14 14 14 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
|
||||
Expression
|
||||
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),
|
||||
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)
|
||||
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
|
||||
1 1 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),
|
||||
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)
|
||||
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
|
||||
Sorting
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
select * from sub5 order by key;
|
||||
54 54 50 50 12 12 0
|
||||
64 64 0 0 0 0 1
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
select * from sub5 order by key;
|
||||
Expression
|
||||
Sorting
|
||||
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),
|
||||
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)
|
||||
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
|
||||
1 1 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),
|
||||
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)
|
||||
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
|
||||
Sorting
|
||||
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),
|
||||
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)
|
||||
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
|
||||
1 1 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),
|
||||
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)
|
||||
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
|
||||
Sorting
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
select * from sub5;
|
||||
0 0 0 0 0 0
|
||||
6 6 6 6 0 0
|
||||
8 8 8 8 0 0
|
||||
@ -260,28 +251,19 @@ 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),
|
||||
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)
|
||||
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;
|
||||
Expression
|
||||
select * from sub5;
|
||||
Union
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Union
|
||||
Expression
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
@ -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),
|
||||
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)
|
||||
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 1 1 0 0
|
||||
0 0 3 3 0 0
|
||||
@ -313,26 +295,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 sub1 l any left join sub2 r on l.y = r.y),
|
||||
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)
|
||||
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
|
||||
Sorting
|
||||
Union
|
||||
Expression
|
||||
Sorting
|
||||
Expression
|
||||
Join
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
Join
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
@ -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),
|
||||
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)
|
||||
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
|
||||
1 1 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),
|
||||
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)
|
||||
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
|
||||
Sorting
|
||||
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),
|
||||
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
|
||||
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
|
||||
1 1 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),
|
||||
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
|
||||
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
|
||||
Sorting
|
||||
Expression
|
||||
@ -455,7 +430,7 @@ Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
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.
|
||||
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
|
||||
1 1 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
|
||||
14 14 14 14 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
|
||||
Expression
|
||||
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),
|
||||
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)
|
||||
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
|
||||
1 1 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),
|
||||
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)
|
||||
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
|
||||
Sorting
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
select * from sub5 order by key;
|
||||
54 54 50 50 12 12 0
|
||||
64 64 0 0 0 0 1
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
select * from sub5 order by key;
|
||||
Expression
|
||||
Sorting
|
||||
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),
|
||||
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)
|
||||
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
|
||||
1 1 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),
|
||||
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)
|
||||
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
|
||||
Sorting
|
||||
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),
|
||||
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)
|
||||
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
|
||||
1 1 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),
|
||||
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)
|
||||
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
|
||||
Sorting
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
select * from sub5;
|
||||
0 0 0 0 0 0
|
||||
6 6 6 6 0 0
|
||||
8 8 8 8 0 0
|
||||
@ -720,25 +686,18 @@ 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),
|
||||
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)
|
||||
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;
|
||||
Expression
|
||||
select * from sub5;
|
||||
Union
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
@ -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),
|
||||
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)
|
||||
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 1 1 0 0
|
||||
0 0 3 3 0 0
|
||||
@ -773,26 +732,22 @@ 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),
|
||||
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)
|
||||
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
|
||||
Sorting
|
||||
Union
|
||||
Expression
|
||||
Sorting
|
||||
Expression
|
||||
Join
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
Join
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
@ -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),
|
||||
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)
|
||||
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
|
||||
1 1 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),
|
||||
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)
|
||||
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
|
||||
Sorting
|
||||
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),
|
||||
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
|
||||
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
|
||||
1 1 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),
|
||||
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
|
||||
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
|
||||
Sorting
|
||||
Expression
|
||||
|
@ -1,16 +1,17 @@
|
||||
drop table if exists tab1;
|
||||
drop table if exists tab2;
|
||||
drop table if exists tab3;
|
||||
drop table if exists tab1 sync;
|
||||
drop table if exists tab2 sync;
|
||||
drop table if exists tab3 sync;
|
||||
|
||||
create table tab1 (x UInt32, y UInt32, shard UInt32) engine = MergeTree order by shard;
|
||||
create table tab2 (y UInt32, z UInt32) engine = MergeTree order by tuple();
|
||||
create table tab3 (z UInt32, a UInt32) engine = MergeTree order by tuple();
|
||||
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 = ReplicatedMergeTree('/clickhouse/tables/{database}/test_02967/tab2', 'r1') 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 tab2 select number * 2, number * 2 from numbers(8);
|
||||
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] -%}
|
||||
|
||||
@ -19,8 +20,9 @@ set parallel_replicas_local_plan=1;
|
||||
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.
|
||||
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;
|
||||
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;
|
||||
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;
|
||||
|
||||
--
|
||||
-- The same query with cte;
|
||||
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),
|
||||
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)
|
||||
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
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
|
||||
--
|
||||
-- GROUP BY should work up to WithMergableStage
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
select * from sub5 order by key;
|
||||
|
||||
explain description=0
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
select * from sub5 order by key;
|
||||
|
||||
--
|
||||
-- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage
|
||||
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),
|
||||
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)
|
||||
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
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
|
||||
--
|
||||
-- ORDER BY in sub1 : sub1 -> WithMergableStage
|
||||
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),
|
||||
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)
|
||||
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
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
|
||||
--
|
||||
-- RIGHT JOIN in sub3: sub3 -> WithMergableStage
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
select * from sub5;
|
||||
|
||||
explain description=0
|
||||
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),
|
||||
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)
|
||||
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;
|
||||
select * from sub5;
|
||||
|
||||
--
|
||||
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage
|
||||
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),
|
||||
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)
|
||||
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
|
||||
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),
|
||||
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)
|
||||
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
|
||||
@ -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),
|
||||
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)
|
||||
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
|
||||
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),
|
||||
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)
|
||||
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 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),
|
||||
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
|
||||
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
|
||||
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),
|
||||
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
|
||||
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 %}
|
||||
|
@ -2,6 +2,7 @@
|
||||
SET enable_analyzer=1;
|
||||
SET distributed_foreground_insert=1;
|
||||
|
||||
DROP TABLE IF EXISTS first_table_lr SYNC;
|
||||
CREATE TABLE first_table_lr
|
||||
(
|
||||
id String,
|
||||
@ -11,6 +12,7 @@ ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_03080/alter', '
|
||||
ORDER BY id;
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS first_table;
|
||||
CREATE TABLE first_table
|
||||
(
|
||||
id String,
|
||||
@ -19,6 +21,7 @@ CREATE TABLE first_table
|
||||
ENGINE = Distributed('test_shard_localhost', currentDatabase(), 'first_table_lr');
|
||||
|
||||
|
||||
DROP TABLE IF EXISTS second_table_lr;
|
||||
CREATE TABLE second_table_lr
|
||||
(
|
||||
id String,
|
||||
@ -26,6 +29,7 @@ CREATE TABLE second_table_lr
|
||||
) ENGINE = MergeTree()
|
||||
ORDER BY id;
|
||||
|
||||
DROP TABLE IF EXISTS second_table;
|
||||
CREATE TABLE second_table
|
||||
(
|
||||
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 second_table VALUES ('1', '2'), ('3', '4');
|
||||
|
||||
DROP TABLE IF EXISTS two_tables;
|
||||
CREATE TABLE two_tables
|
||||
(
|
||||
id String,
|
||||
|
@ -6,12 +6,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
DROP TABLE IF EXISTS ids;
|
||||
CREATE TABLE ids (id UUID, whatever String) Engine=MergeTree ORDER BY tuple();
|
||||
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();
|
||||
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();
|
||||
INSERT INTO data2 VALUES ('a1451105-722e-4fe7-bfaa-65ad2ae249c2', '2000-01-02', 'CREATED');
|
||||
"
|
||||
|
273
tests/queries/0_stateless/03254_pr_join_on_dups.reference
Normal file
273
tests/queries/0_stateless/03254_pr_join_on_dups.reference
Normal 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
|
73
tests/queries/0_stateless/03254_pr_join_on_dups.sql
Normal file
73
tests/queries/0_stateless/03254_pr_join_on_dups.sql
Normal 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;
|
16
tests/queries/0_stateless/03261_pr_semi_anti_join.reference
Normal file
16
tests/queries/0_stateless/03261_pr_semi_anti_join.reference
Normal 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
|
26
tests/queries/0_stateless/03261_pr_semi_anti_join.sql
Normal file
26
tests/queries/0_stateless/03261_pr_semi_anti_join.sql
Normal 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;
|
@ -1,2 +1,2 @@
|
||||
Condition: (_CAST(toDate(ts)) in (-Inf, 1703980800])
|
||||
Granules: 3/3
|
||||
Condition: and((materialize(auid) in [1, 1]), (_CAST(toDate(ts)) in (-Inf, 1703980800]))
|
||||
Granules: 1/3
|
||||
|
91
tests/queries/0_stateless/03269_explain_unique_ids.reference
Normal file
91
tests/queries/0_stateless/03269_explain_unique_ids.reference
Normal 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')
|
60
tests/queries/0_stateless/03269_explain_unique_ids.sh
Executable file
60
tests/queries/0_stateless/03269_explain_unique_ids.sh
Executable 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;
|
||||
"
|
Loading…
Reference in New Issue
Block a user