mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge branch 'master' of github.com:ClickHouse/ClickHouse into tmp_files_lookup
This commit is contained in:
commit
856a66d45f
@ -923,6 +923,9 @@ class IColumn;
|
||||
M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \
|
||||
M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \
|
||||
M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \
|
||||
M(Int32, join_to_sort_minimum_perkey_rows, 40, "The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys", 0) \
|
||||
M(Int32, join_to_sort_maximum_table_rows, 10000, "The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join.", 0) \
|
||||
M(Bool, allow_experimental_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join.", 0) \
|
||||
M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \
|
||||
M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\
|
||||
\
|
||||
@ -945,6 +948,7 @@ class IColumn;
|
||||
M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \
|
||||
M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \
|
||||
M(Bool, allow_archive_path_syntax, true, "File/S3 engines/table function will parse paths with '::' as '<archive> :: <file>' if archive has correct extension", 0) \
|
||||
M(Bool, parallel_replicas_local_plan, false, "Build local plan for local replica", 0) \
|
||||
\
|
||||
M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \
|
||||
M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \
|
||||
|
@ -79,6 +79,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
{"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."},
|
||||
{"database_replicated_allow_replicated_engine_arguments", 1, 0, "Don't allow explicit arguments by default"},
|
||||
{"database_replicated_allow_explicit_uuid", 0, 0, "Added a new setting to disallow explicitly specifying table UUID"},
|
||||
{"parallel_replicas_local_plan", false, false, "Use local plan for local replica in a query with parallel replicas"},
|
||||
}
|
||||
},
|
||||
{"24.8",
|
||||
@ -99,7 +100,10 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
{"use_json_alias_for_old_object_type", true, false, "Use JSON type alias to create new JSON type"},
|
||||
{"type_json_skip_duplicated_paths", false, false, "Allow to skip duplicated paths during JSON parsing"},
|
||||
{"allow_experimental_vector_similarity_index", false, false, "Added new setting to allow experimental vector similarity indexes"},
|
||||
{"input_format_try_infer_datetimes_only_datetime64", true, false, "Allow to infer DateTime instead of DateTime64 in data formats"}
|
||||
{"input_format_try_infer_datetimes_only_datetime64", true, false, "Allow to infer DateTime instead of DateTime64 in data formats"},
|
||||
{"join_to_sort_minimum_perkey_rows", 0, 40, "The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys"},
|
||||
{"join_to_sort_maximum_table_rows", 0, 10000, "The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join."},
|
||||
{"allow_experimental_join_right_table_sorting", false, false, "If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join."},
|
||||
}
|
||||
},
|
||||
{"24.7",
|
||||
|
@ -28,6 +28,7 @@
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/StorageSnapshot.h>
|
||||
#include <Storages/buildQueryTreeForShard.h>
|
||||
#include <Processors/QueryPlan/ParallelReplicasLocalPlan.h>
|
||||
#include <Storages/getStructureOfRemoteTable.h>
|
||||
|
||||
namespace DB
|
||||
@ -38,6 +39,7 @@ namespace ErrorCodes
|
||||
extern const int TOO_LARGE_DISTRIBUTED_DEPTH;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int UNEXPECTED_CLUSTER;
|
||||
extern const int INCONSISTENT_CLUSTER_DEFINITION;
|
||||
}
|
||||
|
||||
namespace ClusterProxy
|
||||
@ -439,7 +441,8 @@ void executeQueryWithParallelReplicas(
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
const ASTPtr & query_ast,
|
||||
ContextPtr context,
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits)
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits,
|
||||
QueryPlanStepPtr analyzed_read_from_merge_tree)
|
||||
{
|
||||
auto logger = getLogger("executeQueryWithParallelReplicas");
|
||||
LOG_DEBUG(logger, "Executing read from {}, header {}, query ({}), stage {} with parallel replicas",
|
||||
@ -485,12 +488,12 @@ void executeQueryWithParallelReplicas(
|
||||
shard_num = column->getUInt(0);
|
||||
}
|
||||
|
||||
const auto shard_count = not_optimized_cluster->getShardCount();
|
||||
ClusterPtr new_cluster = not_optimized_cluster;
|
||||
/// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard
|
||||
/// shards are numbered in order of appearance in the cluster config
|
||||
if (shard_num > 0)
|
||||
{
|
||||
const auto shard_count = not_optimized_cluster->getShardCount();
|
||||
if (shard_num > shard_count)
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
@ -516,21 +519,147 @@ void executeQueryWithParallelReplicas(
|
||||
"`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard");
|
||||
}
|
||||
|
||||
auto external_tables = new_context->getExternalTables();
|
||||
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
|
||||
query_ast,
|
||||
new_cluster,
|
||||
storage_id,
|
||||
header,
|
||||
processed_stage,
|
||||
new_context,
|
||||
getThrottler(new_context),
|
||||
std::move(scalars),
|
||||
std::move(external_tables),
|
||||
getLogger("ReadFromParallelRemoteReplicasStep"),
|
||||
std::move(storage_limits));
|
||||
const auto & shard = new_cluster->getShardsInfo().at(0);
|
||||
size_t max_replicas_to_use = settings.max_parallel_replicas;
|
||||
if (max_replicas_to_use > shard.getAllNodeCount())
|
||||
{
|
||||
LOG_INFO(
|
||||
getLogger("ReadFromParallelRemoteReplicasStep"),
|
||||
"The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "
|
||||
"Will use the latter number to execute the query.",
|
||||
settings.max_parallel_replicas,
|
||||
shard.getAllNodeCount());
|
||||
max_replicas_to_use = shard.getAllNodeCount();
|
||||
}
|
||||
|
||||
query_plan.addStep(std::move(read_from_remote));
|
||||
auto coordinator = std::make_shared<ParallelReplicasReadingCoordinator>(max_replicas_to_use, settings.parallel_replicas_mark_segment_size);
|
||||
|
||||
auto external_tables = new_context->getExternalTables();
|
||||
|
||||
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool> shuffled_pool;
|
||||
if (max_replicas_to_use < shard.getAllNodeCount())
|
||||
{
|
||||
// will be shuffled according to `load_balancing` setting
|
||||
shuffled_pool = shard.pool->getShuffledPools(settings);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If all replicas in cluster are used for query execution,
|
||||
/// try to preserve replicas order as in cluster definition.
|
||||
/// It's important for data locality during query execution
|
||||
/// independently of the query initiator
|
||||
auto priority_func = [](size_t i) { return Priority{static_cast<Int64>(i)}; };
|
||||
shuffled_pool = shard.pool->getShuffledPools(settings, priority_func);
|
||||
}
|
||||
|
||||
std::vector<ConnectionPoolPtr> pools_to_use;
|
||||
pools_to_use.reserve(shuffled_pool.size());
|
||||
for (auto & pool : shuffled_pool)
|
||||
pools_to_use.emplace_back(std::move(pool.pool));
|
||||
|
||||
/// do not build local plan for distributed queries for now (address it later)
|
||||
if (settings.allow_experimental_analyzer && settings.parallel_replicas_local_plan && !shard_num)
|
||||
{
|
||||
/// find local replica index in pool
|
||||
std::optional<size_t> local_replica_index;
|
||||
for (size_t i = 0, s = pools_to_use.size(); i < s; ++i)
|
||||
{
|
||||
const auto & hostname = pools_to_use[i]->getHost();
|
||||
const auto found = std::find_if(
|
||||
begin(shard.local_addresses),
|
||||
end(shard.local_addresses),
|
||||
[&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; });
|
||||
if (found != shard.local_addresses.end())
|
||||
{
|
||||
local_replica_index = i;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!local_replica_index)
|
||||
throw Exception(
|
||||
ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION,
|
||||
"Local replica is not found in '{}' cluster definition, see 'cluster_for_parallel_replicas' setting",
|
||||
new_cluster->getName());
|
||||
|
||||
// resize the pool but keep local replicas in it (and update its index)
|
||||
chassert(max_replicas_to_use <= pools_to_use.size());
|
||||
if (local_replica_index >= max_replicas_to_use)
|
||||
{
|
||||
std::swap(pools_to_use[max_replicas_to_use - 1], pools_to_use[local_replica_index.value()]);
|
||||
local_replica_index = max_replicas_to_use - 1;
|
||||
}
|
||||
pools_to_use.resize(max_replicas_to_use);
|
||||
|
||||
auto [local_plan, with_parallel_replicas] = createLocalPlanForParallelReplicas(
|
||||
query_ast,
|
||||
header,
|
||||
new_context,
|
||||
processed_stage,
|
||||
coordinator,
|
||||
std::move(analyzed_read_from_merge_tree),
|
||||
local_replica_index.value());
|
||||
|
||||
if (!with_parallel_replicas)
|
||||
{
|
||||
query_plan = std::move(*local_plan);
|
||||
return;
|
||||
}
|
||||
|
||||
LOG_DEBUG(logger, "Local replica got replica number {}", local_replica_index.value());
|
||||
|
||||
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
|
||||
query_ast,
|
||||
new_cluster,
|
||||
storage_id,
|
||||
coordinator,
|
||||
header,
|
||||
processed_stage,
|
||||
new_context,
|
||||
getThrottler(new_context),
|
||||
std::move(scalars),
|
||||
std::move(external_tables),
|
||||
getLogger("ReadFromParallelRemoteReplicasStep"),
|
||||
std::move(storage_limits),
|
||||
std::move(pools_to_use),
|
||||
local_replica_index);
|
||||
|
||||
auto remote_plan = std::make_unique<QueryPlan>();
|
||||
remote_plan->addStep(std::move(read_from_remote));
|
||||
|
||||
DataStreams input_streams;
|
||||
input_streams.reserve(2);
|
||||
input_streams.emplace_back(local_plan->getCurrentDataStream());
|
||||
input_streams.emplace_back(remote_plan->getCurrentDataStream());
|
||||
|
||||
std::vector<QueryPlanPtr> plans;
|
||||
plans.emplace_back(std::move(local_plan));
|
||||
plans.emplace_back(std::move(remote_plan));
|
||||
|
||||
auto union_step = std::make_unique<UnionStep>(std::move(input_streams));
|
||||
query_plan.unitePlans(std::move(union_step), std::move(plans));
|
||||
}
|
||||
else
|
||||
{
|
||||
chassert(max_replicas_to_use <= pools_to_use.size());
|
||||
pools_to_use.resize(max_replicas_to_use);
|
||||
|
||||
auto read_from_remote = std::make_unique<ReadFromParallelRemoteReplicasStep>(
|
||||
query_ast,
|
||||
new_cluster,
|
||||
storage_id,
|
||||
std::move(coordinator),
|
||||
header,
|
||||
processed_stage,
|
||||
new_context,
|
||||
getThrottler(new_context),
|
||||
std::move(scalars),
|
||||
std::move(external_tables),
|
||||
getLogger("ReadFromParallelRemoteReplicasStep"),
|
||||
std::move(storage_limits),
|
||||
std::move(pools_to_use));
|
||||
|
||||
query_plan.addStep(std::move(read_from_remote));
|
||||
}
|
||||
}
|
||||
|
||||
void executeQueryWithParallelReplicas(
|
||||
@ -540,7 +669,8 @@ void executeQueryWithParallelReplicas(
|
||||
const QueryTreeNodePtr & query_tree,
|
||||
const PlannerContextPtr & planner_context,
|
||||
ContextPtr context,
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits)
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits,
|
||||
QueryPlanStepPtr analyzed_read_from_merge_tree)
|
||||
{
|
||||
QueryTreeNodePtr modified_query_tree = query_tree->clone();
|
||||
rewriteJoinToGlobalJoin(modified_query_tree, context);
|
||||
@ -550,7 +680,8 @@ void executeQueryWithParallelReplicas(
|
||||
= InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_tree, context, SelectQueryOptions(processed_stage).analyze());
|
||||
auto modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree);
|
||||
|
||||
executeQueryWithParallelReplicas(query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits);
|
||||
executeQueryWithParallelReplicas(
|
||||
query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits, std::move(analyzed_read_from_merge_tree));
|
||||
}
|
||||
|
||||
void executeQueryWithParallelReplicas(
|
||||
@ -646,6 +777,49 @@ void executeQueryWithParallelReplicasCustomKey(
|
||||
context, query_info.query, storage_id.getDatabaseName(), storage_id.getTableName(), /*table_function_ptr=*/nullptr);
|
||||
executeQueryWithParallelReplicasCustomKey(query_plan, storage_id, query_info, columns, snapshot, processed_stage, header, context);
|
||||
}
|
||||
|
||||
bool canUseParallelReplicasOnInitiator(const ContextPtr & context)
|
||||
{
|
||||
if (!context->canUseParallelReplicasOnInitiator())
|
||||
return false;
|
||||
|
||||
auto cluster = context->getClusterForParallelReplicas();
|
||||
if (cluster->getShardCount() == 1)
|
||||
return cluster->getShardsInfo()[0].getAllNodeCount() > 1;
|
||||
|
||||
/// parallel replicas with distributed table
|
||||
auto scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{};
|
||||
UInt64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified
|
||||
const auto it = scalars.find("_shard_num");
|
||||
if (it != scalars.end())
|
||||
{
|
||||
const Block & block = it->second;
|
||||
const auto & column = block.safeGetByPosition(0).column;
|
||||
shard_num = column->getUInt(0);
|
||||
}
|
||||
if (shard_num > 0)
|
||||
{
|
||||
const auto shard_count = cluster->getShardCount();
|
||||
if (shard_num > shard_count)
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Shard number is greater than shard count: shard_num={} shard_count={} cluster={}",
|
||||
shard_num,
|
||||
shard_count,
|
||||
cluster->getName());
|
||||
|
||||
return cluster->getShardsInfo().at(shard_num - 1).getAllNodeCount() > 1;
|
||||
}
|
||||
|
||||
if (cluster->getShardCount() > 1)
|
||||
throw DB::Exception(
|
||||
ErrorCodes::UNEXPECTED_CLUSTER,
|
||||
"`cluster_for_parallel_replicas` setting refers to cluster with {} shards. Expected a cluster with one shard",
|
||||
cluster->getShardCount());
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -35,6 +35,9 @@ using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
|
||||
class PlannerContext;
|
||||
using PlannerContextPtr = std::shared_ptr<PlannerContext>;
|
||||
|
||||
class IQueryPlanStep;
|
||||
using QueryPlanStepPtr = std::unique_ptr<IQueryPlanStep>;
|
||||
|
||||
namespace ClusterProxy
|
||||
{
|
||||
|
||||
@ -55,6 +58,8 @@ using AdditionalShardFilterGenerator = std::function<ASTPtr(uint64_t)>;
|
||||
AdditionalShardFilterGenerator
|
||||
getShardFilterGeneratorForCustomKey(const Cluster & cluster, ContextPtr context, const ColumnsDescription & columns);
|
||||
|
||||
bool canUseParallelReplicasOnInitiator(const ContextPtr & context);
|
||||
|
||||
/// Execute a distributed query, creating a query plan, from which the query pipeline can be built.
|
||||
/// `stream_factory` object encapsulates the logic of creating plans for a different type of query
|
||||
/// (currently SELECT, DESCRIBE).
|
||||
@ -81,7 +86,8 @@ void executeQueryWithParallelReplicas(
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
const ASTPtr & query_ast,
|
||||
ContextPtr context,
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits);
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits,
|
||||
QueryPlanStepPtr read_from_merge_tree = nullptr);
|
||||
|
||||
void executeQueryWithParallelReplicas(
|
||||
QueryPlan & query_plan,
|
||||
@ -98,7 +104,8 @@ void executeQueryWithParallelReplicas(
|
||||
const QueryTreeNodePtr & query_tree,
|
||||
const PlannerContextPtr & planner_context,
|
||||
ContextPtr context,
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits);
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits,
|
||||
QueryPlanStepPtr read_from_merge_tree);
|
||||
|
||||
void executeQueryWithParallelReplicasCustomKey(
|
||||
QueryPlan & query_plan,
|
||||
|
@ -34,6 +34,23 @@ void AddedColumns<true>::buildOutput()
|
||||
{
|
||||
if (join_data_avg_perkey_rows < output_by_row_list_threshold)
|
||||
buildOutputFromBlocks<true>();
|
||||
else if (join_data_sorted)
|
||||
{
|
||||
for (size_t i = 0; i < this->size(); ++i)
|
||||
{
|
||||
auto & col = columns[i];
|
||||
for (auto row_ref_i : lazy_output.row_refs)
|
||||
{
|
||||
if (row_ref_i)
|
||||
{
|
||||
const RowRefList * row_ref_list = reinterpret_cast<const RowRefList *>(row_ref_i);
|
||||
col->insertRangeFrom(*row_ref_list->block->getByPosition(right_indexes[i]).column, row_ref_list->row_num, row_ref_list->rows);
|
||||
}
|
||||
else
|
||||
type_name[i].type->insertDefaultInto(*col);
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < this->size(); ++i)
|
||||
|
@ -66,6 +66,9 @@ public:
|
||||
, join_on_keys(join_on_keys_)
|
||||
, additional_filter_expression(additional_filter_expression_)
|
||||
, rows_to_add(left_block.rows())
|
||||
, join_data_avg_perkey_rows(join.getJoinedData()->avgPerKeyRows())
|
||||
, output_by_row_list_threshold(join.getTableJoin().outputByRowListPerkeyRowsThreshold())
|
||||
, join_data_sorted(join.getJoinedData()->sorted)
|
||||
, is_join_get(is_join_get_)
|
||||
{
|
||||
size_t num_columns_to_add = block_with_columns_to_add.columns();
|
||||
@ -113,8 +116,6 @@ public:
|
||||
if (columns[j]->isNullable() && !saved_column->isNullable())
|
||||
nullable_column_ptrs[j] = typeid_cast<ColumnNullable *>(columns[j].get());
|
||||
}
|
||||
join_data_avg_perkey_rows = join.getJoinedData()->avgPerKeyRows();
|
||||
output_by_row_list_threshold = join.getTableJoin().outputByRowListPerkeyRowsThreshold();
|
||||
}
|
||||
|
||||
size_t size() const { return columns.size(); }
|
||||
@ -149,6 +150,7 @@ public:
|
||||
bool output_by_row_list = false;
|
||||
size_t join_data_avg_perkey_rows = 0;
|
||||
size_t output_by_row_list_threshold = 0;
|
||||
bool join_data_sorted = false;
|
||||
IColumn::Filter filter;
|
||||
|
||||
void reserve(bool need_replicate)
|
||||
|
@ -649,7 +649,6 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
|
||||
}
|
||||
data->keys_to_join = total_rows;
|
||||
shrinkStoredBlocksToFit(total_bytes);
|
||||
|
||||
return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
|
||||
}
|
||||
|
||||
@ -1361,4 +1360,96 @@ bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table
|
||||
return false;
|
||||
}
|
||||
|
||||
template <JoinKind KIND, typename Map, JoinStrictness STRICTNESS>
|
||||
void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]])
|
||||
{
|
||||
constexpr JoinFeatures<KIND, STRICTNESS, Map> join_features;
|
||||
if constexpr (!join_features.is_all_join || (!join_features.left && !join_features.inner))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Only left or inner join table can be reranged.");
|
||||
else
|
||||
{
|
||||
auto merge_rows_into_one_block = [&](BlocksList & blocks, RowRefList & rows_ref)
|
||||
{
|
||||
auto it = rows_ref.begin();
|
||||
if (it.ok())
|
||||
{
|
||||
if (blocks.empty() || blocks.back().rows() >= DEFAULT_BLOCK_SIZE)
|
||||
blocks.emplace_back(it->block->cloneEmpty());
|
||||
}
|
||||
else
|
||||
{
|
||||
return;
|
||||
}
|
||||
auto & block = blocks.back();
|
||||
size_t start_row = block.rows();
|
||||
for (; it.ok(); ++it)
|
||||
{
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
{
|
||||
auto & col = block.getByPosition(i).column->assumeMutableRef();
|
||||
col.insertFrom(*it->block->getByPosition(i).column, it->row_num);
|
||||
}
|
||||
}
|
||||
if (block.rows() > start_row)
|
||||
{
|
||||
RowRefList new_rows_ref(&block, start_row, block.rows() - start_row);
|
||||
rows_ref = std::move(new_rows_ref);
|
||||
}
|
||||
};
|
||||
|
||||
auto visit_rows_map = [&](BlocksList & blocks, MapsAll & rows_map)
|
||||
{
|
||||
switch (data->type)
|
||||
{
|
||||
#define M(TYPE) \
|
||||
case Type::TYPE: \
|
||||
{\
|
||||
rows_map.TYPE->forEachMapped([&](RowRefList & rows_ref) { merge_rows_into_one_block(blocks, rows_ref); }); \
|
||||
break; \
|
||||
}
|
||||
APPLY_FOR_JOIN_VARIANTS(M)
|
||||
#undef M
|
||||
default:
|
||||
break;
|
||||
}
|
||||
};
|
||||
BlocksList sorted_blocks;
|
||||
visit_rows_map(sorted_blocks, map);
|
||||
data->blocks.swap(sorted_blocks);
|
||||
}
|
||||
}
|
||||
|
||||
void HashJoin::tryRerangeRightTableData()
|
||||
{
|
||||
if (!table_join->allowJoinSorting() || table_join->getMixedJoinExpression() || !isInnerOrLeft(kind) || strictness != JoinStrictness::All)
|
||||
return;
|
||||
|
||||
/// We should not rerange the right table on such conditions:
|
||||
/// 1. the right table is already reranged by key or it is empty.
|
||||
/// 2. the join clauses size is greater than 1, like `...join on a.key1=b.key1 or a.key2=b.key2`, we can not rerange the right table on different set of keys.
|
||||
/// 3. the number of right table rows exceed the threshold, which may result in a significant cost for reranging and lead to performance degradation.
|
||||
/// 4. the keys of right table is very sparse, which may result in insignificant performance improvement after reranging by key.
|
||||
if (!data || data->sorted || data->blocks.empty() || data->maps.size() > 1 || data->rows_to_join > table_join->sortRightMaximumTableRows() || data->avgPerKeyRows() < table_join->sortRightMinimumPerkeyRows())
|
||||
return;
|
||||
|
||||
if (data->keys_to_join == 0)
|
||||
data->keys_to_join = getTotalRowCount();
|
||||
|
||||
/// If the there is no columns to add, means no columns to output, then the rerange would not improve performance by using column's `insertRangeFrom`
|
||||
/// to replace column's `insertFrom` to make the output.
|
||||
if (sample_block_with_columns_to_add.columns() == 0)
|
||||
{
|
||||
LOG_DEBUG(log, "The joined right table total rows :{}, total keys :{}", data->rows_to_join, data->keys_to_join);
|
||||
return;
|
||||
}
|
||||
[[maybe_unused]] bool result = joinDispatch(
|
||||
kind,
|
||||
strictness,
|
||||
data->maps.front(),
|
||||
/*prefer_use_maps_all*/ false,
|
||||
[&](auto kind_, auto strictness_, auto & map_) { tryRerangeRightTableDataImpl<kind_, decltype(map_), strictness_>(map_); });
|
||||
chassert(result);
|
||||
data->sorted = true;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -345,11 +345,12 @@ public:
|
||||
|
||||
size_t blocks_allocated_size = 0;
|
||||
size_t blocks_nullmaps_allocated_size = 0;
|
||||
|
||||
/// Number of rows of right table to join
|
||||
size_t rows_to_join = 0;
|
||||
/// Number of keys of right table to join
|
||||
size_t keys_to_join = 0;
|
||||
/// Whether the right table reranged by key
|
||||
bool sorted = false;
|
||||
|
||||
size_t avgPerKeyRows() const
|
||||
{
|
||||
@ -465,6 +466,10 @@ private:
|
||||
|
||||
void validateAdditionalFilterExpression(std::shared_ptr<ExpressionActions> additional_filter_expression);
|
||||
bool needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const;
|
||||
|
||||
void tryRerangeRightTableData() override;
|
||||
template <JoinKind KIND, typename Map, JoinStrictness STRICTNESS>
|
||||
void tryRerangeRightTableDataImpl(Map & map);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -83,7 +83,6 @@ public:
|
||||
const Block & block_with_columns_to_add,
|
||||
const MapsTemplateVector & maps_,
|
||||
bool is_join_get = false);
|
||||
|
||||
private:
|
||||
template <typename KeyGetter, bool is_asof_join>
|
||||
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes);
|
||||
@ -199,4 +198,3 @@ extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Semi, Hash
|
||||
extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Anti, HashJoin::MapsOne>;
|
||||
extern template class HashJoinMethods<JoinKind::Full, JoinStrictness::Asof, HashJoin::MapsAsof>;
|
||||
}
|
||||
|
||||
|
@ -115,6 +115,7 @@ public:
|
||||
/// Peek next stream of delayed joined blocks.
|
||||
virtual IBlocksStreamPtr getDelayedBlocks() { return nullptr; }
|
||||
virtual bool hasDelayedBlocks() const { return false; }
|
||||
virtual void tryRerangeRightTableData() {}
|
||||
|
||||
virtual IBlocksStreamPtr
|
||||
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0;
|
||||
|
@ -123,6 +123,7 @@ struct RowRefList : RowRef
|
||||
|
||||
RowRefList() {} /// NOLINT
|
||||
RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_), rows(1) {}
|
||||
RowRefList(const Block * block_, size_t row_start_, size_t rows_) : RowRef(block_, row_start_), rows(static_cast<SizeT>(rows_)) {}
|
||||
|
||||
ForwardIterator begin() const { return ForwardIterator(this); }
|
||||
|
||||
|
@ -45,7 +45,7 @@ Chunk Squashing::squash(Chunk && input_chunk)
|
||||
|
||||
Chunk Squashing::add(Chunk && input_chunk)
|
||||
{
|
||||
if (!input_chunk)
|
||||
if (!input_chunk || input_chunk.getNumRows() == 0)
|
||||
return {};
|
||||
|
||||
/// Just read block is already enough.
|
||||
|
@ -116,6 +116,9 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary
|
||||
, max_files_to_merge(settings.join_on_disk_max_files_to_merge)
|
||||
, temporary_files_codec(settings.temporary_files_codec)
|
||||
, output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold)
|
||||
, sort_right_minimum_perkey_rows(settings.join_to_sort_minimum_perkey_rows)
|
||||
, sort_right_maximum_table_rows(settings.join_to_sort_maximum_table_rows)
|
||||
, allow_join_sorting(settings.allow_experimental_join_right_table_sorting)
|
||||
, max_memory_usage(settings.max_memory_usage)
|
||||
, tmp_volume(tmp_volume_)
|
||||
, tmp_data(tmp_data_)
|
||||
|
@ -149,6 +149,9 @@ private:
|
||||
const size_t max_files_to_merge = 0;
|
||||
const String temporary_files_codec = "LZ4";
|
||||
const size_t output_by_rowlist_perkey_rows_threshold = 0;
|
||||
const size_t sort_right_minimum_perkey_rows = 0;
|
||||
const size_t sort_right_maximum_table_rows = 0;
|
||||
const bool allow_join_sorting = false;
|
||||
|
||||
/// Value if setting max_memory_usage for query, can be used when max_bytes_in_join is not specified.
|
||||
size_t max_memory_usage = 0;
|
||||
@ -297,6 +300,9 @@ public:
|
||||
}
|
||||
|
||||
size_t outputByRowListPerkeyRowsThreshold() const { return output_by_rowlist_perkey_rows_threshold; }
|
||||
size_t sortRightMinimumPerkeyRows() const { return sort_right_minimum_perkey_rows; }
|
||||
size_t sortRightMaximumTableRows() const { return sort_right_maximum_table_rows; }
|
||||
bool allowJoinSorting() const { return allow_join_sorting; }
|
||||
size_t defaultMaxBytes() const { return default_max_bytes; }
|
||||
size_t maxJoinedBlockRows() const { return max_joined_block_rows; }
|
||||
size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; }
|
||||
|
@ -893,31 +893,6 @@ 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))
|
||||
{
|
||||
// (1) find read step
|
||||
QueryPlan::Node * node = query_plan.getRootNode();
|
||||
ReadFromMergeTree * reading = nullptr;
|
||||
while (node)
|
||||
{
|
||||
reading = typeid_cast<ReadFromMergeTree *>(node->step.get());
|
||||
if (reading)
|
||||
break;
|
||||
|
||||
QueryPlan::Node * prev_node = node;
|
||||
if (!node->children.empty())
|
||||
{
|
||||
chassert(node->children.size() == 1);
|
||||
node = node->children.at(0);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Step is expected to be ReadFromMergeTree but it's {}",
|
||||
prev_node->step->getName());
|
||||
}
|
||||
}
|
||||
|
||||
chassert(reading);
|
||||
if (query_context->canUseParallelReplicasCustomKey() && query_context->getClientInfo().distributed_depth == 0)
|
||||
{
|
||||
if (auto cluster = query_context->getClusterForParallelReplicas();
|
||||
@ -940,21 +915,50 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
query_plan = std::move(query_plan_parallel_replicas);
|
||||
}
|
||||
}
|
||||
else if (query_context->canUseParallelReplicasOnInitiator())
|
||||
else if (ClusterProxy::canUseParallelReplicasOnInitiator(query_context))
|
||||
{
|
||||
// (1) find read step
|
||||
QueryPlan::Node * node = query_plan.getRootNode();
|
||||
ReadFromMergeTree * reading = nullptr;
|
||||
while (node)
|
||||
{
|
||||
reading = typeid_cast<ReadFromMergeTree *>(node->step.get());
|
||||
if (reading)
|
||||
break;
|
||||
|
||||
QueryPlan::Node * prev_node = node;
|
||||
if (!node->children.empty())
|
||||
{
|
||||
chassert(node->children.size() == 1);
|
||||
node = node->children.at(0);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Step is expected to be ReadFromMergeTree but it's {}",
|
||||
prev_node->step->getName());
|
||||
}
|
||||
}
|
||||
|
||||
chassert(reading);
|
||||
|
||||
// (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read
|
||||
if (settings.parallel_replicas_min_number_of_rows_per_replica > 0)
|
||||
{
|
||||
auto result_ptr = reading->selectRangesToRead();
|
||||
|
||||
UInt64 rows_to_read = result_ptr->selected_rows;
|
||||
|
||||
reading->setAnalyzedResult(std::move(result_ptr));
|
||||
|
||||
if (table_expression_query_info.trivial_limit > 0 && table_expression_query_info.trivial_limit < rows_to_read)
|
||||
rows_to_read = table_expression_query_info.trivial_limit;
|
||||
|
||||
if (max_block_size_limited && (max_block_size_limited < rows_to_read))
|
||||
rows_to_read = max_block_size_limited;
|
||||
|
||||
const size_t number_of_replicas_to_use = rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica;
|
||||
const size_t number_of_replicas_to_use
|
||||
= rows_to_read / settings.parallel_replicas_min_number_of_rows_per_replica;
|
||||
LOG_TRACE(
|
||||
getLogger("Planner"),
|
||||
"Estimated {} rows to read. It is enough work for {} parallel replicas",
|
||||
@ -980,6 +984,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
{
|
||||
from_stage = QueryProcessingStage::WithMergeableState;
|
||||
QueryPlan query_plan_parallel_replicas;
|
||||
QueryPlanStepPtr reading_step = std::move(node->step);
|
||||
ClusterProxy::executeQueryWithParallelReplicas(
|
||||
query_plan_parallel_replicas,
|
||||
storage->getStorageID(),
|
||||
@ -987,9 +992,24 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres
|
||||
table_expression_query_info.query_tree,
|
||||
table_expression_query_info.planner_context,
|
||||
query_context,
|
||||
table_expression_query_info.storage_limits);
|
||||
table_expression_query_info.storage_limits,
|
||||
std::move(reading_step));
|
||||
query_plan = std::move(query_plan_parallel_replicas);
|
||||
}
|
||||
else
|
||||
{
|
||||
QueryPlan query_plan_no_parallel_replicas;
|
||||
storage->read(
|
||||
query_plan_no_parallel_replicas,
|
||||
columns_names,
|
||||
storage_snapshot,
|
||||
table_expression_query_info,
|
||||
query_context,
|
||||
from_stage,
|
||||
max_block_size,
|
||||
max_streams);
|
||||
query_plan = std::move(query_plan_no_parallel_replicas);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -52,7 +52,13 @@ std::stack<const QueryNode *> getSupportingParallelReplicasQuery(const IQueryTre
|
||||
const auto & storage = table_node.getStorage();
|
||||
/// Here we check StorageDummy as well, to support a query tree with replaced storages.
|
||||
if (std::dynamic_pointer_cast<MergeTreeData>(storage) || typeid_cast<const StorageDummy *>(storage.get()))
|
||||
{
|
||||
/// parallel replicas is not supported with FINAL
|
||||
if (table_node.getTableExpressionModifiers() && table_node.getTableExpressionModifiers()->hasFinal())
|
||||
return {};
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
32
src/Processors/QueryPlan/ConvertingActions.cpp
Normal file
32
src/Processors/QueryPlan/ConvertingActions.cpp
Normal file
@ -0,0 +1,32 @@
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects)
|
||||
{
|
||||
if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header))
|
||||
return;
|
||||
|
||||
auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name;
|
||||
|
||||
auto get_converting_dag = [mode](const Block & block_, const Block & header_)
|
||||
{
|
||||
/// Convert header structure to expected.
|
||||
/// Also we ignore constants from result and replace it with constants from header.
|
||||
/// It is needed for functions like `now64()` or `randConstant()` because their values may be different.
|
||||
return ActionsDAG::makeConvertingActions(
|
||||
block_.getColumnsWithTypeAndName(),
|
||||
header_.getColumnsWithTypeAndName(),
|
||||
mode,
|
||||
true);
|
||||
};
|
||||
|
||||
auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header);
|
||||
auto converting = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(convert_actions_dag));
|
||||
plan.addStep(std::move(converting));
|
||||
}
|
||||
|
||||
}
|
9
src/Processors/QueryPlan/ConvertingActions.h
Normal file
9
src/Processors/QueryPlan/ConvertingActions.h
Normal file
@ -0,0 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class QueryPlan;
|
||||
class Block;
|
||||
|
||||
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects);
|
||||
}
|
@ -2,43 +2,13 @@
|
||||
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
#include <Interpreters/InterpreterSelectQuery.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
#include <Processors/QueryPlan/ConvertingActions.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects)
|
||||
{
|
||||
if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header))
|
||||
return;
|
||||
|
||||
auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name;
|
||||
|
||||
auto get_converting_dag = [mode](const Block & block_, const Block & header_)
|
||||
{
|
||||
/// Convert header structure to expected.
|
||||
/// Also we ignore constants from result and replace it with constants from header.
|
||||
/// It is needed for functions like `now64()` or `randConstant()` because their values may be different.
|
||||
return ActionsDAG::makeConvertingActions(
|
||||
block_.getColumnsWithTypeAndName(),
|
||||
header_.getColumnsWithTypeAndName(),
|
||||
mode,
|
||||
true);
|
||||
};
|
||||
|
||||
auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header);
|
||||
auto converting = std::make_unique<ExpressionStep>(plan.getCurrentDataStream(), std::move(convert_actions_dag));
|
||||
plan.addStep(std::move(converting));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
std::unique_ptr<QueryPlan> createLocalPlan(
|
||||
const ASTPtr & query_ast,
|
||||
const Block & header,
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
|
||||
#include <Processors/QueryPlan/Optimizations/actionsDAGUtils.h>
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
#include <Processors/QueryPlan/ReadFromRemote.h>
|
||||
#include <Processors/QueryPlan/SortingStep.h>
|
||||
#include <Processors/QueryPlan/TotalsHavingStep.h>
|
||||
#include <Processors/QueryPlan/UnionStep.h>
|
||||
@ -899,6 +900,18 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl
|
||||
return {};
|
||||
}
|
||||
|
||||
static bool readingFromParallelReplicas(const QueryPlan::Node * node)
|
||||
{
|
||||
IQueryPlanStep * step = node->step.get();
|
||||
while (!node->children.empty())
|
||||
{
|
||||
step = node->children.front()->step.get();
|
||||
node = node->children.front();
|
||||
}
|
||||
|
||||
return typeid_cast<const ReadFromParallelRemoteReplicasStep *>(step);
|
||||
}
|
||||
|
||||
void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes)
|
||||
{
|
||||
if (node.children.size() != 1)
|
||||
@ -924,6 +937,16 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes)
|
||||
std::vector<InputOrderInfoPtr> infos;
|
||||
infos.reserve(node.children.size());
|
||||
|
||||
for (const auto * child : union_node->children)
|
||||
{
|
||||
/// in case of parallel replicas
|
||||
/// avoid applying read-in-order optimization for local replica
|
||||
/// since it will lead to different parallel replicas modes
|
||||
/// between local and remote nodes
|
||||
if (readingFromParallelReplicas(child))
|
||||
return;
|
||||
}
|
||||
|
||||
for (auto * child : union_node->children)
|
||||
{
|
||||
infos.push_back(buildInputOrderInfo(*sorting, *child, steps_to_update));
|
||||
|
97
src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp
Normal file
97
src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp
Normal file
@ -0,0 +1,97 @@
|
||||
#include <Processors/QueryPlan/ParallelReplicasLocalPlan.h>
|
||||
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Interpreters/ActionsDAG.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Processors/QueryPlan/ConvertingActions.h>
|
||||
#include <Processors/QueryPlan/ExpressionStep.h>
|
||||
#include <Processors/QueryPlan/ISourceStep.h>
|
||||
#include <Processors/QueryPlan/ReadFromMergeTree.h>
|
||||
#include <Processors/Sources/NullSource.h>
|
||||
#include <Processors/Transforms/ExpressionTransform.h>
|
||||
#include <Processors/Transforms/FilterTransform.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataSelectExecutor.h>
|
||||
#include <Storages/MergeTree/RequestResponse.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::pair<std::unique_ptr<QueryPlan>, bool> createLocalPlanForParallelReplicas(
|
||||
const ASTPtr & query_ast,
|
||||
const Block & header,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
ParallelReplicasReadingCoordinatorPtr coordinator,
|
||||
QueryPlanStepPtr analyzed_read_from_merge_tree,
|
||||
size_t replica_number)
|
||||
{
|
||||
checkStackSize();
|
||||
|
||||
auto query_plan = std::make_unique<QueryPlan>();
|
||||
auto new_context = Context::createCopy(context);
|
||||
|
||||
/// Do not push down limit to local plan, as it will break `rows_before_limit_at_least` counter.
|
||||
if (processed_stage == QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit)
|
||||
processed_stage = QueryProcessingStage::WithMergeableStateAfterAggregation;
|
||||
|
||||
/// Do not apply AST optimizations, because query
|
||||
/// is already optimized and some optimizations
|
||||
/// can be applied only for non-distributed tables
|
||||
/// and we can produce query, inconsistent with remote plans.
|
||||
auto select_query_options = SelectQueryOptions(processed_stage).ignoreASTOptimizations();
|
||||
|
||||
/// For Analyzer, identifier in GROUP BY/ORDER BY/LIMIT BY lists has been resolved to
|
||||
/// ConstantNode in QueryTree if it is an alias of a constant, so we should not replace
|
||||
/// ConstantNode with ProjectionNode again(https://github.com/ClickHouse/ClickHouse/issues/62289).
|
||||
new_context->setSetting("enable_positional_arguments", Field(false));
|
||||
new_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0));
|
||||
auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options);
|
||||
query_plan = std::make_unique<QueryPlan>(std::move(interpreter).extractQueryPlan());
|
||||
|
||||
QueryPlan::Node * node = query_plan->getRootNode();
|
||||
ReadFromMergeTree * reading = nullptr;
|
||||
while (node)
|
||||
{
|
||||
reading = typeid_cast<ReadFromMergeTree *>(node->step.get());
|
||||
if (reading)
|
||||
break;
|
||||
|
||||
if (!node->children.empty())
|
||||
node = node->children.at(0);
|
||||
else
|
||||
node = nullptr;
|
||||
}
|
||||
|
||||
if (!reading)
|
||||
/// it can happened if merge tree table is empty, - it'll be replaced with ReadFromPreparedSource
|
||||
return {std::move(query_plan), false};
|
||||
|
||||
ReadFromMergeTree::AnalysisResultPtr analyzed_result_ptr;
|
||||
if (analyzed_read_from_merge_tree.get())
|
||||
{
|
||||
auto * analyzed_merge_tree = typeid_cast<ReadFromMergeTree *>(analyzed_read_from_merge_tree.get());
|
||||
if (analyzed_merge_tree)
|
||||
analyzed_result_ptr = analyzed_merge_tree->getAnalyzedResult();
|
||||
}
|
||||
|
||||
MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement announcement)
|
||||
{ coordinator->handleInitialAllRangesAnnouncement(std::move(announcement)); };
|
||||
|
||||
MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional<ParallelReadResponse>
|
||||
{ return coordinator->handleRequest(std::move(req)); };
|
||||
|
||||
auto read_from_merge_tree_parallel_replicas = reading->createLocalParallelReplicasReadingStep(
|
||||
analyzed_result_ptr, std::move(all_ranges_cb), std::move(read_task_cb), replica_number);
|
||||
node->step = std::move(read_from_merge_tree_parallel_replicas);
|
||||
|
||||
addConvertingActions(*query_plan, header, /*has_missing_objects=*/false);
|
||||
|
||||
return {std::move(query_plan), true};
|
||||
}
|
||||
|
||||
}
|
19
src/Processors/QueryPlan/ParallelReplicasLocalPlan.h
Normal file
19
src/Processors/QueryPlan/ParallelReplicasLocalPlan.h
Normal file
@ -0,0 +1,19 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/QueryProcessingStage.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Processors/QueryPlan/QueryPlan.h>
|
||||
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::pair<std::unique_ptr<QueryPlan>, bool> createLocalPlanForParallelReplicas(
|
||||
const ASTPtr & query_ast,
|
||||
const Block & header,
|
||||
ContextPtr context,
|
||||
QueryProcessingStage::Enum processed_stage,
|
||||
ParallelReplicasReadingCoordinatorPtr coordinator,
|
||||
QueryPlanStepPtr read_from_merge_tree,
|
||||
size_t replica_number);
|
||||
}
|
@ -277,7 +277,10 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read_,
|
||||
LoggerPtr log_,
|
||||
AnalysisResultPtr analyzed_result_ptr_,
|
||||
bool enable_parallel_reading)
|
||||
bool enable_parallel_reading_,
|
||||
std::optional<MergeTreeAllRangesCallback> all_ranges_callback_,
|
||||
std::optional<MergeTreeReadTaskCallback> read_task_callback_,
|
||||
std::optional<size_t> number_of_current_replica_)
|
||||
: SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader(
|
||||
storage_snapshot_->getSampleBlockForColumns(all_column_names_),
|
||||
query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_)
|
||||
@ -295,13 +298,21 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
, max_block_numbers_to_read(std::move(max_block_numbers_to_read_))
|
||||
, log(std::move(log_))
|
||||
, analyzed_result_ptr(analyzed_result_ptr_)
|
||||
, is_parallel_reading_from_replicas(enable_parallel_reading)
|
||||
, is_parallel_reading_from_replicas(enable_parallel_reading_)
|
||||
, enable_remove_parts_from_snapshot_optimization(query_info_.merge_tree_enable_remove_parts_from_snapshot_optimization)
|
||||
, number_of_current_replica(number_of_current_replica_)
|
||||
{
|
||||
if (is_parallel_reading_from_replicas)
|
||||
{
|
||||
all_ranges_callback = context->getMergeTreeAllRangesCallback();
|
||||
read_task_callback = context->getMergeTreeReadTaskCallback();
|
||||
if (all_ranges_callback_.has_value())
|
||||
all_ranges_callback = all_ranges_callback_.value();
|
||||
else
|
||||
all_ranges_callback = context->getMergeTreeAllRangesCallback();
|
||||
|
||||
if (read_task_callback_.has_value())
|
||||
read_task_callback = read_task_callback_.value();
|
||||
else
|
||||
read_task_callback = context->getMergeTreeReadTaskCallback();
|
||||
}
|
||||
|
||||
const auto & settings = context->getSettingsRef();
|
||||
@ -335,11 +346,33 @@ ReadFromMergeTree::ReadFromMergeTree(
|
||||
enable_vertical_final);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadFromMergeTree> ReadFromMergeTree::createLocalParallelReplicasReadingStep(
|
||||
AnalysisResultPtr analyzed_result_ptr_,
|
||||
MergeTreeAllRangesCallback all_ranges_callback_,
|
||||
MergeTreeReadTaskCallback read_task_callback_,
|
||||
size_t replica_number)
|
||||
{
|
||||
const bool enable_parallel_reading = true;
|
||||
return std::make_unique<ReadFromMergeTree>(
|
||||
prepared_parts,
|
||||
mutations_snapshot,
|
||||
all_column_names,
|
||||
data,
|
||||
getQueryInfo(),
|
||||
getStorageSnapshot(),
|
||||
getContext(),
|
||||
block_size.max_block_size_rows,
|
||||
requested_num_streams,
|
||||
max_block_numbers_to_read,
|
||||
log,
|
||||
std::move(analyzed_result_ptr_),
|
||||
enable_parallel_reading,
|
||||
all_ranges_callback_,
|
||||
read_task_callback_,
|
||||
replica_number);
|
||||
}
|
||||
|
||||
Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
|
||||
RangesInDataParts parts_with_range,
|
||||
Names required_columns,
|
||||
PoolSettings pool_settings)
|
||||
Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings)
|
||||
{
|
||||
const auto & client_info = context->getClientInfo();
|
||||
|
||||
@ -347,7 +380,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas(
|
||||
{
|
||||
.all_callback = all_ranges_callback.value(),
|
||||
.callback = read_task_callback.value(),
|
||||
.number_of_current_replica = client_info.number_of_current_replica,
|
||||
.number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica),
|
||||
};
|
||||
|
||||
/// We have a special logic for local replica. It has to read less data, because in some cases it should
|
||||
@ -529,7 +562,7 @@ Pipe ReadFromMergeTree::readInOrder(
|
||||
{
|
||||
.all_callback = all_ranges_callback.value(),
|
||||
.callback = read_task_callback.value(),
|
||||
.number_of_current_replica = client_info.number_of_current_replica,
|
||||
.number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica),
|
||||
};
|
||||
|
||||
auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier;
|
||||
@ -584,11 +617,12 @@ Pipe ReadFromMergeTree::readInOrder(
|
||||
context);
|
||||
}
|
||||
|
||||
/// Actually it means that parallel reading from replicas enabled
|
||||
/// and we have to collaborate with initiator.
|
||||
/// In this case we won't set approximate rows, because it will be accounted multiple times.
|
||||
const auto in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0;
|
||||
const bool set_total_rows_approx = !is_parallel_reading_from_replicas;
|
||||
/// If parallel replicas enabled, set total rows in progress here only on initiator with local plan
|
||||
/// Otherwise rows will counted multiple times
|
||||
const UInt64 in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0;
|
||||
const bool parallel_replicas_local_plan_for_initiator = is_parallel_reading_from_replicas
|
||||
&& context->getSettingsRef().parallel_replicas_local_plan && context->canUseParallelReplicasOnInitiator();
|
||||
const bool set_total_rows_approx = !is_parallel_reading_from_replicas || parallel_replicas_local_plan_for_initiator;
|
||||
|
||||
Pipes pipes;
|
||||
for (size_t i = 0; i < parts_with_ranges.size(); ++i)
|
||||
@ -1422,11 +1456,8 @@ static void buildIndexes(
|
||||
|
||||
const auto & settings = context->getSettingsRef();
|
||||
|
||||
indexes.emplace(ReadFromMergeTree::Indexes{{
|
||||
filter_actions_dag,
|
||||
context,
|
||||
primary_key_column_names,
|
||||
primary_key.expression}, {}, {}, {}, {}, false, {}});
|
||||
indexes.emplace(
|
||||
ReadFromMergeTree::Indexes{KeyCondition{filter_actions_dag, context, primary_key_column_names, primary_key.expression}});
|
||||
|
||||
if (metadata_snapshot->hasPartitionKey())
|
||||
{
|
||||
@ -1978,6 +2009,33 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
|
||||
{
|
||||
auto result = getAnalysisResult();
|
||||
|
||||
if (is_parallel_reading_from_replicas && context->canUseParallelReplicasOnInitiator()
|
||||
&& context->getSettingsRef().parallel_replicas_local_plan)
|
||||
{
|
||||
CoordinationMode mode = CoordinationMode::Default;
|
||||
switch (result.read_type)
|
||||
{
|
||||
case ReadFromMergeTree::ReadType::Default:
|
||||
mode = CoordinationMode::Default;
|
||||
break;
|
||||
case ReadFromMergeTree::ReadType::InOrder:
|
||||
mode = CoordinationMode::WithOrder;
|
||||
break;
|
||||
case ReadFromMergeTree::ReadType::InReverseOrder:
|
||||
mode = CoordinationMode::ReverseOrder;
|
||||
break;
|
||||
case ReadFromMergeTree::ReadType::ParallelReplicas:
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Read type can't be ParallelReplicas on initiator");
|
||||
}
|
||||
|
||||
chassert(number_of_current_replica.has_value());
|
||||
chassert(all_ranges_callback.has_value());
|
||||
|
||||
/// initialize working set from local replica
|
||||
all_ranges_callback.value()(
|
||||
InitialAllRangesAnnouncement(mode, result.parts_with_ranges.getDescriptions(), number_of_current_replica.value()));
|
||||
}
|
||||
|
||||
if (enable_remove_parts_from_snapshot_optimization)
|
||||
{
|
||||
/// Do not keep data parts in snapshot.
|
||||
|
@ -121,7 +121,16 @@ public:
|
||||
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read_,
|
||||
LoggerPtr log_,
|
||||
AnalysisResultPtr analyzed_result_ptr_,
|
||||
bool enable_parallel_reading);
|
||||
bool enable_parallel_reading_,
|
||||
std::optional<MergeTreeAllRangesCallback> all_ranges_callback_ = std::nullopt,
|
||||
std::optional<MergeTreeReadTaskCallback> read_task_callback_ = std::nullopt,
|
||||
std::optional<size_t> number_of_current_replica_ = std::nullopt);
|
||||
|
||||
std::unique_ptr<ReadFromMergeTree> createLocalParallelReplicasReadingStep(
|
||||
AnalysisResultPtr analyzed_result_ptr_,
|
||||
MergeTreeAllRangesCallback all_ranges_callback_,
|
||||
MergeTreeReadTaskCallback read_task_callback_,
|
||||
size_t replica_number);
|
||||
|
||||
static constexpr auto name = "ReadFromMergeTree";
|
||||
String getName() const override { return name; }
|
||||
@ -143,6 +152,11 @@ public:
|
||||
|
||||
struct Indexes
|
||||
{
|
||||
explicit Indexes(KeyCondition key_condition_)
|
||||
: key_condition(std::move(key_condition_))
|
||||
, use_skip_indexes(false)
|
||||
{}
|
||||
|
||||
KeyCondition key_condition;
|
||||
std::optional<PartitionPruner> partition_pruner;
|
||||
std::optional<KeyCondition> minmax_idx_condition;
|
||||
@ -268,6 +282,7 @@ private:
|
||||
std::optional<MergeTreeReadTaskCallback> read_task_callback;
|
||||
bool enable_vertical_final = false;
|
||||
bool enable_remove_parts_from_snapshot_optimization = true;
|
||||
std::optional<size_t> number_of_current_replica;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -21,8 +21,8 @@
|
||||
#include <Client/ConnectionPoolWithFailover.h>
|
||||
#include <QueryPipeline/QueryPipelineBuilder.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Storages/MergeTree/ParallelReplicasReadingCoordinator.h>
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
|
||||
#include <fmt/format.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -362,6 +362,7 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
|
||||
ASTPtr query_ast_,
|
||||
ClusterPtr cluster_,
|
||||
const StorageID & storage_id_,
|
||||
ParallelReplicasReadingCoordinatorPtr coordinator_,
|
||||
Block header_,
|
||||
QueryProcessingStage::Enum stage_,
|
||||
ContextMutablePtr context_,
|
||||
@ -369,11 +370,14 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
|
||||
Scalars scalars_,
|
||||
Tables external_tables_,
|
||||
LoggerPtr log_,
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits_)
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits_,
|
||||
std::vector<ConnectionPoolPtr> pools_to_use_,
|
||||
std::optional<size_t> exclude_pool_index_)
|
||||
: ISourceStep(DataStream{.header = std::move(header_)})
|
||||
, cluster(cluster_)
|
||||
, query_ast(query_ast_)
|
||||
, storage_id(storage_id_)
|
||||
, coordinator(std::move(coordinator_))
|
||||
, stage(std::move(stage_))
|
||||
, context(context_)
|
||||
, throttler(throttler_)
|
||||
@ -381,16 +385,24 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep(
|
||||
, external_tables{external_tables_}
|
||||
, storage_limits(std::move(storage_limits_))
|
||||
, log(log_)
|
||||
, pools_to_use(std::move(pools_to_use_))
|
||||
, exclude_pool_index(exclude_pool_index_)
|
||||
{
|
||||
chassert(cluster->getShardCount() == 1);
|
||||
|
||||
std::vector<String> description;
|
||||
description.push_back(fmt::format("query: {}", formattedAST(query_ast)));
|
||||
std::vector<String> replicas;
|
||||
replicas.reserve(pools_to_use.size());
|
||||
|
||||
for (const auto & pool : cluster->getShardsInfo().front().per_replica_pools)
|
||||
description.push_back(fmt::format("Replica: {}", pool->getHost()));
|
||||
for (size_t i = 0, l = pools_to_use.size(); i < l; ++i)
|
||||
{
|
||||
if (exclude_pool_index.has_value() && i == exclude_pool_index)
|
||||
continue;
|
||||
|
||||
setStepDescription(boost::algorithm::join(description, ", "));
|
||||
replicas.push_back(pools_to_use[i]->getAddress());
|
||||
}
|
||||
|
||||
auto description = fmt::format("Query: {} Replicas: {}", formattedAST(query_ast), fmt::join(replicas, ", "));
|
||||
setStepDescription(std::move(description));
|
||||
}
|
||||
|
||||
void ReadFromParallelRemoteReplicasStep::enforceSorting(SortDescription output_sort_description)
|
||||
@ -406,48 +418,29 @@ void ReadFromParallelRemoteReplicasStep::enforceAggregationInOrder()
|
||||
void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
Pipes pipes;
|
||||
const Settings & current_settings = context->getSettingsRef();
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings);
|
||||
|
||||
const auto & shard = cluster->getShardsInfo().at(0);
|
||||
size_t max_replicas_to_use = current_settings.max_parallel_replicas;
|
||||
if (max_replicas_to_use > shard.getAllNodeCount())
|
||||
std::vector<std::string_view> addresses;
|
||||
addresses.reserve(pools_to_use.size());
|
||||
for (size_t i = 0, l = pools_to_use.size(); i < l; ++i)
|
||||
{
|
||||
LOG_INFO(
|
||||
getLogger("ReadFromParallelRemoteReplicasStep"),
|
||||
"The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). "
|
||||
"Will use the latter number to execute the query.",
|
||||
current_settings.max_parallel_replicas,
|
||||
shard.getAllNodeCount());
|
||||
max_replicas_to_use = shard.getAllNodeCount();
|
||||
if (exclude_pool_index.has_value() && i == exclude_pool_index)
|
||||
continue;
|
||||
|
||||
addresses.emplace_back(pools_to_use[i]->getAddress());
|
||||
}
|
||||
LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", fmt::join(addresses, ", "));
|
||||
|
||||
std::vector<ConnectionPoolWithFailover::Base::ShuffledPool> shuffled_pool;
|
||||
if (max_replicas_to_use < shard.getAllNodeCount())
|
||||
for (size_t i = 0, l = pools_to_use.size(); i < l; ++i)
|
||||
{
|
||||
shuffled_pool = shard.pool->getShuffledPools(current_settings);
|
||||
shuffled_pool.resize(max_replicas_to_use);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// try to preserve replicas order if all replicas in cluster are used for query execution
|
||||
/// it's important for data locality during query execution
|
||||
auto priority_func = [](size_t i) { return Priority{static_cast<Int64>(i)}; };
|
||||
shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func);
|
||||
}
|
||||
if (exclude_pool_index.has_value() && i == exclude_pool_index)
|
||||
continue;
|
||||
|
||||
coordinator
|
||||
= std::make_shared<ParallelReplicasReadingCoordinator>(max_replicas_to_use, current_settings.parallel_replicas_mark_segment_size);
|
||||
|
||||
for (size_t i=0; i < max_replicas_to_use; ++i)
|
||||
{
|
||||
IConnections::ReplicaInfo replica_info
|
||||
{
|
||||
IConnections::ReplicaInfo replica_info{
|
||||
/// we should use this number specifically because efficiency of data distribution by consistent hash depends on it.
|
||||
.number_of_current_replica = i,
|
||||
};
|
||||
|
||||
addPipeForSingeReplica(pipes, shuffled_pool[i].pool, replica_info);
|
||||
addPipeForSingeReplica(pipes, pools_to_use[i], replica_info);
|
||||
}
|
||||
|
||||
auto pipe = Pipe::unitePipes(std::move(pipes));
|
||||
|
@ -70,6 +70,7 @@ public:
|
||||
ASTPtr query_ast_,
|
||||
ClusterPtr cluster_,
|
||||
const StorageID & storage_id_,
|
||||
ParallelReplicasReadingCoordinatorPtr coordinator_,
|
||||
Block header_,
|
||||
QueryProcessingStage::Enum stage_,
|
||||
ContextMutablePtr context_,
|
||||
@ -77,7 +78,9 @@ public:
|
||||
Scalars scalars_,
|
||||
Tables external_tables_,
|
||||
LoggerPtr log_,
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits_);
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits_,
|
||||
std::vector<ConnectionPoolPtr> pools_to_use,
|
||||
std::optional<size_t> exclude_pool_index_ = std::nullopt);
|
||||
|
||||
String getName() const override { return "ReadFromRemoteParallelReplicas"; }
|
||||
|
||||
@ -100,6 +103,8 @@ private:
|
||||
Tables external_tables;
|
||||
std::shared_ptr<const StorageLimitsList> storage_limits;
|
||||
LoggerPtr log;
|
||||
std::vector<ConnectionPoolPtr> pools_to_use;
|
||||
std::optional<size_t> exclude_pool_index;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -299,13 +299,17 @@ IProcessor::Status FillingRightJoinSideTransform::prepare()
|
||||
|
||||
void FillingRightJoinSideTransform::work()
|
||||
{
|
||||
auto block = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns());
|
||||
auto & input = inputs.front();
|
||||
auto block = input.getHeader().cloneWithColumns(chunk.detachColumns());
|
||||
|
||||
if (for_totals)
|
||||
join->setTotals(block);
|
||||
else
|
||||
stop_reading = !join->addBlockToJoin(block);
|
||||
|
||||
if (input.isFinished())
|
||||
join->tryRerangeRightTableData();
|
||||
|
||||
set_totals = for_totals;
|
||||
}
|
||||
|
||||
|
@ -185,7 +185,8 @@ public:
|
||||
void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
||||
{
|
||||
if (++sent_initial_requests > replicas_count)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Initiator received more initial requests than there are replicas");
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR, "Initiator received more initial requests than there are replicas: replica_num={}", announcement.replica_num);
|
||||
|
||||
doHandleInitialAllRangesAnnouncement(std::move(announcement));
|
||||
}
|
||||
@ -433,9 +434,9 @@ void DefaultCoordinator::setProgressCallback()
|
||||
|
||||
void DefaultCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
||||
{
|
||||
const auto replica_num = announcement.replica_num;
|
||||
LOG_DEBUG(log, "Initial request: {}", announcement.describe());
|
||||
|
||||
LOG_DEBUG(log, "Initial request from replica {}: {}", announcement.replica_num, announcement.describe());
|
||||
const auto replica_num = announcement.replica_num;
|
||||
|
||||
initializeReadingState(std::move(announcement));
|
||||
|
||||
@ -839,6 +840,7 @@ public:
|
||||
|
||||
Parts all_parts_to_read;
|
||||
size_t total_rows_to_read = 0;
|
||||
bool state_initialized{false};
|
||||
|
||||
LoggerPtr log = getLogger(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator"));
|
||||
};
|
||||
@ -858,7 +860,9 @@ void InOrderCoordinator<mode>::markReplicaAsUnavailable(size_t replica_number)
|
||||
template <CoordinationMode mode>
|
||||
void InOrderCoordinator<mode>::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement)
|
||||
{
|
||||
LOG_TRACE(log, "Received an announcement {}", announcement.describe());
|
||||
LOG_TRACE(log, "Received an announcement : {}", announcement.describe());
|
||||
|
||||
++stats[announcement.replica_num].number_of_requests;
|
||||
|
||||
size_t new_rows_to_read = 0;
|
||||
|
||||
@ -868,13 +872,16 @@ void InOrderCoordinator<mode>::doHandleInitialAllRangesAnnouncement(InitialAllRa
|
||||
auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
|
||||
[&part] (const Part & other) { return other.description.info == part.info; });
|
||||
|
||||
/// We have the same part - add the info about presence on current replica to it
|
||||
/// We have the same part - add the info about presence on the corresponding replica to it
|
||||
if (the_same_it != all_parts_to_read.end())
|
||||
{
|
||||
the_same_it->replicas.insert(announcement.replica_num);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (state_initialized)
|
||||
continue;
|
||||
|
||||
auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(),
|
||||
[&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); });
|
||||
|
||||
@ -889,9 +896,10 @@ void InOrderCoordinator<mode>::doHandleInitialAllRangesAnnouncement(InitialAllRa
|
||||
std::sort(ranges.begin(), ranges.end());
|
||||
}
|
||||
|
||||
++stats[announcement.replica_num].number_of_requests;
|
||||
state_initialized = true;
|
||||
|
||||
if (new_rows_to_read > 0)
|
||||
// progress_callback is not set when local plan is used for initiator
|
||||
if (progress_callback && new_rows_to_read > 0)
|
||||
{
|
||||
Progress progress;
|
||||
progress.total_rows_to_read = new_rows_to_read;
|
||||
@ -911,7 +919,7 @@ ParallelReadResponse InOrderCoordinator<mode>::handleRequest(ParallelReadRequest
|
||||
"Replica {} decided to read in {} mode, not in {}. This is a bug",
|
||||
request.replica_num, magic_enum::enum_name(request.mode), magic_enum::enum_name(mode));
|
||||
|
||||
LOG_TRACE(log, "Got request from replica {}, data {}", request.replica_num, request.describe());
|
||||
LOG_TRACE(log, "Got read request: {}", request.describe());
|
||||
|
||||
ParallelReadResponse response;
|
||||
response.description = request.description;
|
||||
@ -925,8 +933,15 @@ ParallelReadResponse InOrderCoordinator<mode>::handleRequest(ParallelReadRequest
|
||||
if (global_part_it == all_parts_to_read.end())
|
||||
continue;
|
||||
|
||||
if (global_part_it->replicas.empty())
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Part {} requested by replica {} is not registered in working set",
|
||||
part.info.getPartNameV1(),
|
||||
request.replica_num);
|
||||
|
||||
if (!global_part_it->replicas.contains(request.replica_num))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} doesn't exist on replica {} according to the global state", part.info.getPartNameV1(), request.replica_num);
|
||||
continue;
|
||||
|
||||
size_t current_mark_size = 0;
|
||||
|
||||
@ -1057,6 +1072,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode)
|
||||
break;
|
||||
}
|
||||
|
||||
// progress_callback is not set when local plan is used for initiator
|
||||
if (progress_callback)
|
||||
pimpl->setProgressCallback(std::move(progress_callback));
|
||||
|
||||
|
@ -44,9 +44,7 @@ void ParallelReadRequest::serialize(WriteBuffer & out) const
|
||||
|
||||
String ParallelReadRequest::describe() const
|
||||
{
|
||||
String result;
|
||||
result += fmt::format("replica_num: {} \n", replica_num);
|
||||
result += fmt::format("min_num_of_marks: {} \n", min_number_of_marks);
|
||||
String result = fmt::format("replica_num {}, min_num_of_marks {}, ", replica_num, min_number_of_marks);
|
||||
result += description.describe();
|
||||
return result;
|
||||
}
|
||||
@ -131,10 +129,7 @@ void InitialAllRangesAnnouncement::serialize(WriteBuffer & out) const
|
||||
|
||||
String InitialAllRangesAnnouncement::describe()
|
||||
{
|
||||
String result;
|
||||
result += description.describe();
|
||||
result += fmt::format("----------\nReceived from {} replica\n", replica_num);
|
||||
return result;
|
||||
return fmt::format("replica {}, mode {}, {}", replica_num, mode, description.describe());
|
||||
}
|
||||
|
||||
InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in)
|
||||
|
@ -473,7 +473,7 @@ class CI:
|
||||
JobNames.STATELESS_TEST_FLAKY_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties(
|
||||
required_builds=[BuildNames.PACKAGE_ASAN],
|
||||
pr_only=True,
|
||||
timeout=3600,
|
||||
timeout=3 * 3600,
|
||||
# TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf
|
||||
# reference_job_name=JobNames.STATELESS_TEST_RELEASE,
|
||||
),
|
||||
|
@ -9,6 +9,7 @@ from urllib.parse import quote
|
||||
from unidiff import PatchSet # type: ignore
|
||||
|
||||
from build_download_helper import get_gh_api
|
||||
from ci_config import Labels
|
||||
from env_helper import (
|
||||
GITHUB_EVENT_PATH,
|
||||
GITHUB_REPOSITORY,
|
||||
@ -16,7 +17,6 @@ from env_helper import (
|
||||
GITHUB_SERVER_URL,
|
||||
GITHUB_UPSTREAM_REPOSITORY,
|
||||
)
|
||||
from ci_config import Labels
|
||||
from get_robot_token import get_best_robot_token
|
||||
from github_helper import GitHub
|
||||
|
||||
@ -459,16 +459,18 @@ class PRInfo:
|
||||
sync_repo = gh.get_repo(GITHUB_REPOSITORY)
|
||||
sync_pr = sync_repo.get_pull(self.number)
|
||||
# Find the commit that is in both repos, upstream and cloud
|
||||
sync_commits = sync_pr.get_commits().reversed
|
||||
upstream_commits = upstream_pr.get_commits().reversed
|
||||
# Do not ever use `reversed` here, otherwise the list of commits is not full
|
||||
sync_commits = list(sync_pr.get_commits())
|
||||
upstream_commits = list(upstream_pr.get_commits())
|
||||
# Github objects are compared by _url attribute. We can't compare them directly and
|
||||
# should compare commits by SHA1
|
||||
upstream_shas = [c.sha for c in upstream_commits]
|
||||
logging.info("Commits in upstream PR:\n %s", ", ".join(upstream_shas))
|
||||
sync_shas = [c.sha for c in sync_commits]
|
||||
logging.info("Commits in sync PR:\n %s", ", ".join(reversed(sync_shas)))
|
||||
logging.info("Commits in sync PR:\n %s", ", ".join(sync_shas))
|
||||
|
||||
# find latest synced commit
|
||||
# find latest synced commit, search from the latest
|
||||
upstream_commits.reverse()
|
||||
last_synced_upstream_commit = None
|
||||
for commit in upstream_commits:
|
||||
if commit.sha in sync_shas:
|
||||
|
@ -916,6 +916,7 @@ class SettingsRandomizer:
|
||||
"min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]),
|
||||
"max_parsing_threads": lambda: random.choice([0, 1, 10]),
|
||||
"optimize_functions_to_subcolumns": lambda: random.randint(0, 1),
|
||||
"parallel_replicas_local_plan": lambda: random.randint(0, 1),
|
||||
}
|
||||
|
||||
@staticmethod
|
||||
|
@ -115,10 +115,6 @@
|
||||
<parallel_replicas>
|
||||
<shard>
|
||||
<internal_replication>false</internal_replication>
|
||||
<replica>
|
||||
<host>127.0.0.1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>127.0.0.2</host>
|
||||
<port>9000</port>
|
||||
@ -147,6 +143,10 @@
|
||||
<host>127.0.0.8</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>127.0.0.1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>127.0.0.9</host>
|
||||
<port>9000</port>
|
||||
|
@ -49,5 +49,6 @@ def test_skip_all_replicas(
|
||||
"max_parallel_replicas": max_parallel_replicas,
|
||||
"cluster_for_parallel_replicas": cluster_name,
|
||||
"skip_unavailable_shards": skip_unavailable_shards,
|
||||
"parallel_replicas_local_plan": 0,
|
||||
},
|
||||
)
|
||||
|
@ -100,7 +100,7 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off
|
||||
504 2000
|
||||
1000000
|
||||
-- { echoOn } --
|
||||
explain pipeline select a from pr_t group by a order by a limit 5 offset 500;
|
||||
explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings parallel_replicas_local_plan=0;
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Limit)
|
||||
@ -119,6 +119,31 @@ ExpressionTransform
|
||||
Resize 1 → 4
|
||||
GroupingAggregatedTransform 3 → 1
|
||||
(ReadFromRemoteParallelReplicas)
|
||||
explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1;
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Limit)
|
||||
Limit
|
||||
(Sorting)
|
||||
MergingSortedTransform 4 → 1
|
||||
MergeSortingTransform × 4
|
||||
LimitsCheckingTransform × 4
|
||||
PartialSortingTransform × 4
|
||||
(Expression)
|
||||
ExpressionTransform × 4
|
||||
(MergingAggregated)
|
||||
MergingAggregatedBucketTransform × 4
|
||||
Resize 1 → 4
|
||||
FinishAggregatingInOrderTransform 3 → 1
|
||||
(Union)
|
||||
(Aggregating)
|
||||
FinalizeAggregatedTransform
|
||||
AggregatingInOrderTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeSelect(pool: ReadPoolParallelReplicasInOrder, algorithm: InOrder) 0 → 1
|
||||
(ReadFromRemoteParallelReplicas)
|
||||
select a, count() from pr_t group by a order by a limit 5 offset 500;
|
||||
500 1000
|
||||
501 1000
|
||||
|
@ -66,14 +66,14 @@ insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6);
|
||||
set allow_experimental_parallel_reading_from_replicas = 1;
|
||||
set parallel_replicas_for_non_replicated_merge_tree = 1;
|
||||
set max_parallel_replicas = 3;
|
||||
set use_hedged_requests = 0;
|
||||
set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost';
|
||||
set distributed_aggregation_memory_efficient=1;
|
||||
|
||||
select count() from pr_t;
|
||||
|
||||
-- { echoOn } --
|
||||
explain pipeline select a from pr_t group by a order by a limit 5 offset 500;
|
||||
explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings parallel_replicas_local_plan=0;
|
||||
explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1;
|
||||
|
||||
select a, count() from pr_t group by a order by a limit 5 offset 500;
|
||||
select a, count() from pr_t group by a, b order by a limit 5 offset 500;
|
||||
|
@ -31,7 +31,7 @@
|
||||
29 2j&S)ba?XG QuQj 17163829389637435056
|
||||
3 UlI+1 14144472852965836438
|
||||
=============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE ===============
|
||||
0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\')
|
||||
0 2 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\')
|
||||
0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10
|
||||
1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0;
|
||||
1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=1;
|
||||
@ -58,7 +58,7 @@ U c 10
|
||||
UlI+1 10
|
||||
bX?}ix [ Ny]2 G 10
|
||||
t<iT X48q:Z]t0 10
|
||||
0 3 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`key` AS `key`, `__table3`.`value1` AS `value1`, `__table3`.`value2` AS `value2` FROM `default`.`join_inner_table` AS `__table3` PREWHERE (`__table3`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table3`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2`
|
||||
0 2 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`key` AS `key`, `__table3`.`value1` AS `value1`, `__table3`.`value2` AS `value2` FROM `default`.`join_inner_table` AS `__table3` PREWHERE (`__table3`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table3`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2`
|
||||
0 3 SELECT `key`, `value1`, `value2` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2`
|
||||
0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_` USING (`key`) GROUP BY `key`, `value1`, `value2`
|
||||
1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, enable_analyzer=0;
|
||||
|
@ -22,8 +22,8 @@ SELECT
|
||||
LIMIT 100;
|
||||
|
||||
SET max_parallel_replicas = 3;
|
||||
SET prefer_localhost_replica = 1;
|
||||
SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost';
|
||||
SET parallel_replicas_local_plan = 1;
|
||||
SET joined_subquery_requires_alias = 0;
|
||||
|
||||
SELECT '=============== INNER QUERY (NO PARALLEL) ===============';
|
||||
|
@ -4,7 +4,9 @@ INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10);
|
||||
|
||||
SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1;
|
||||
SET send_logs_level='error';
|
||||
SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79';
|
||||
-- with local plan for initiator, the query can be executed fast on initator, we can simply not come to the point where unavailable replica can be detected
|
||||
-- therefore disable local plan for now
|
||||
SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79', parallel_replicas_local_plan=0;
|
||||
|
||||
SYSTEM FLUSH LOGS;
|
||||
|
||||
|
@ -9,4 +9,4 @@
|
||||
7885388429666205427
|
||||
8124171311239967992
|
||||
1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n enable_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1;
|
||||
0 2 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1`
|
||||
0 1 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1`
|
||||
|
@ -1,5 +1,5 @@
|
||||
-- Tags: zookeeper
|
||||
|
||||
DROP TABLE IF EXISTS join_inner_table__fuzz_146_replicated;
|
||||
CREATE TABLE join_inner_table__fuzz_146_replicated
|
||||
(
|
||||
`id` UUID,
|
||||
@ -17,6 +17,8 @@ INSERT INTO join_inner_table__fuzz_146_replicated
|
||||
SELECT CAST('833c9e22-c245-4eb5-8745-117a9a1f26b1', 'UUID') AS id, CAST(rowNumberInAllBlocks(), 'String') AS key, *
|
||||
FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 10;
|
||||
|
||||
SET parallel_replicas_local_plan = 1;
|
||||
|
||||
-- Simple query with analyzer and pure parallel replicas
|
||||
SELECT number
|
||||
FROM join_inner_table__fuzz_146_replicated
|
||||
@ -49,3 +51,5 @@ WHERE
|
||||
)
|
||||
GROUP BY is_initial_query, query
|
||||
ORDER BY is_initial_query DESC, c, query;
|
||||
|
||||
DROP TABLE join_inner_table__fuzz_146_replicated;
|
||||
|
@ -6,16 +6,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"}
|
||||
|
||||
function were_parallel_replicas_used () {
|
||||
# Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it
|
||||
$CLICKHOUSE_CLIENT --query "
|
||||
SELECT
|
||||
initial_query_id,
|
||||
concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used
|
||||
concat('Used parallel replicas: ', (ProfileEvents['ParallelReplicasUsedCount'] > 0)::bool::String) as used
|
||||
FROM system.query_log
|
||||
WHERE event_date >= yesterday()
|
||||
AND initial_query_id LIKE '$1%'
|
||||
GROUP BY initial_query_id
|
||||
ORDER BY min(event_time_microseconds) ASC
|
||||
AND query_id = initial_query_id
|
||||
AND type = 'QueryFinish'
|
||||
AND current_database = '$CLICKHOUSE_DATABASE'
|
||||
ORDER BY event_time_microseconds ASC
|
||||
FORMAT TSV"
|
||||
}
|
||||
|
||||
@ -48,11 +49,11 @@ function run_query_with_pure_parallel_replicas () {
|
||||
--query "$3" \
|
||||
--query_id "${1}_pure" \
|
||||
--max_parallel_replicas 3 \
|
||||
--prefer_localhost_replica 1 \
|
||||
--cluster_for_parallel_replicas "parallel_replicas" \
|
||||
--allow_experimental_parallel_reading_from_replicas 1 \
|
||||
--parallel_replicas_for_non_replicated_merge_tree 1 \
|
||||
--parallel_replicas_min_number_of_rows_per_replica "$2" \
|
||||
--max_threads 5 \
|
||||
|& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }'
|
||||
}
|
||||
|
||||
|
@ -4,8 +4,8 @@
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 10 estimated parallel replicas
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 3 estimated parallel replicas
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 33 estimated parallel replicas
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_0_pure Distinct parallel subqueries: 2 Used parallel replicas: true
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Distinct parallel subqueries: 0 Used parallel replicas: false
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Distinct parallel subqueries: 1 Used parallel replicas: true
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Distinct parallel subqueries: 1 Used parallel replicas: true
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Distinct parallel subqueries: 2 Used parallel replicas: true
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_0_pure Used parallel replicas: true
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Used parallel replicas: false
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Used parallel replicas: true
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Used parallel replicas: true
|
||||
02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Used parallel replicas: true
|
||||
|
@ -9,17 +9,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"}
|
||||
|
||||
function were_parallel_replicas_used () {
|
||||
# Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it
|
||||
$CLICKHOUSE_CLIENT --query "
|
||||
SELECT
|
||||
initial_query_id,
|
||||
concat('Distinct parallel subqueries: ' , countDistinctIf(query, initial_query_id != query_id)::String) as subqueries_parallelized,
|
||||
concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used
|
||||
concat('Used parallel replicas: ', (ProfileEvents['ParallelReplicasUsedCount'] > 0)::bool::String) as used
|
||||
FROM system.query_log
|
||||
WHERE event_date >= yesterday()
|
||||
AND initial_query_id LIKE '$1%'
|
||||
GROUP BY initial_query_id
|
||||
ORDER BY min(event_time_microseconds) ASC
|
||||
AND query_id = initial_query_id
|
||||
AND type = 'QueryFinish'
|
||||
AND current_database = '$CLICKHOUSE_DATABASE'
|
||||
ORDER BY event_time_microseconds ASC
|
||||
FORMAT TSV"
|
||||
}
|
||||
|
||||
|
@ -27,6 +27,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE replicas_summary (n Int64) ENGINE = Mer
|
||||
|
||||
query_id_base="02841_summary_$CLICKHOUSE_DATABASE"
|
||||
|
||||
# TODO: rethink the test, for now temporary disable parallel_replicas_local_plan
|
||||
echo "
|
||||
SELECT *
|
||||
FROM replicas_summary
|
||||
@ -36,7 +37,8 @@ echo "
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost',
|
||||
allow_experimental_parallel_reading_from_replicas = 2,
|
||||
parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
interactive_delay=0
|
||||
interactive_delay=0,
|
||||
parallel_replicas_local_plan=0
|
||||
"\
|
||||
| ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_0" --data-binary @- -vvv 2>&1 \
|
||||
| grep "Summary" | grep -cv '"read_rows":"0"'
|
||||
@ -50,7 +52,8 @@ echo "
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost',
|
||||
allow_experimental_parallel_reading_from_replicas = 2,
|
||||
parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
interactive_delay=99999999999
|
||||
interactive_delay=99999999999,
|
||||
parallel_replicas_local_plan=0
|
||||
"\
|
||||
| ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_high" --data-binary @- -vvv 2>&1 \
|
||||
| grep "Summary" | grep -cv '"read_rows":"0"'
|
||||
|
@ -26,7 +26,8 @@ WHERE query_id in (select query_id from system.query_log where current_database
|
||||
AND message LIKE '%Total rows to read: 3000%' SETTINGS allow_experimental_parallel_reading_from_replicas=0;
|
||||
|
||||
-- reading in order coordinator
|
||||
SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b';
|
||||
-- disable parallel_replicas_local_plan since the test relay on traces which only present in case of no local plan
|
||||
SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, parallel_replicas_local_plan=0, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b';
|
||||
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT count() > 0 FROM system.text_log
|
||||
|
@ -1,4 +1,4 @@
|
||||
100 4950
|
||||
10000 49995000
|
||||
1
|
||||
89
|
||||
90
|
||||
@ -6,3 +6,9 @@
|
||||
92
|
||||
93
|
||||
1
|
||||
93
|
||||
92
|
||||
91
|
||||
90
|
||||
89
|
||||
1
|
||||
|
@ -2,11 +2,12 @@ DROP TABLE IF EXISTS test;
|
||||
|
||||
CREATE TABLE test (k UInt64, v String)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY k;
|
||||
ORDER BY k
|
||||
SETTINGS index_granularity=1;
|
||||
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(10_000);
|
||||
|
||||
SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost';
|
||||
SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost';
|
||||
|
||||
-- default coordinator
|
||||
SELECT count(), sum(k)
|
||||
@ -14,12 +15,18 @@ FROM test
|
||||
SETTINGS log_comment = '02950_parallel_replicas_used_replicas_count';
|
||||
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0;
|
||||
SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0;
|
||||
|
||||
-- In order coordinator
|
||||
SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2';
|
||||
SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2', merge_tree_min_rows_for_concurrent_read=1, max_threads=1;
|
||||
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0;
|
||||
SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0;
|
||||
|
||||
-- In reverse order coordinator
|
||||
SELECT k FROM test order by k desc limit 5 offset 9906 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_3', merge_tree_min_rows_for_concurrent_read=1, max_threads=1;
|
||||
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_3' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0;
|
||||
|
||||
DROP TABLE test;
|
||||
|
@ -11,10 +11,8 @@ nested join with analyzer and parallel replicas, both global
|
||||
420336 420336 420336 -140112
|
||||
420378 420378 420378 -140126
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
|
||||
<Debug> DefaultCoordinator: Coordination done
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
|
||||
<Debug> DefaultCoordinator: Coordination done
|
||||
|
||||
nested join with analyzer and parallel replicas, global + local
|
||||
@ -29,10 +27,8 @@ nested join with analyzer and parallel replicas, global + local
|
||||
420336 420336 420336 -140112
|
||||
420378 420378 420378 -140126
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
|
||||
<Debug> DefaultCoordinator: Coordination done
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
|
||||
<Debug> DefaultCoordinator: Coordination done
|
||||
|
||||
nested join with analyzer and parallel replicas, both local, both full sorting merge join
|
||||
@ -47,10 +43,8 @@ nested join with analyzer and parallel replicas, both local, both full sorting m
|
||||
420336 420336 420336 -140112
|
||||
420378 420378 420378 -140126
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
|
||||
<Debug> WithOrderCoordinator: Coordination done
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
|
||||
<Debug> WithOrderCoordinator: Coordination done
|
||||
|
||||
nested join with analyzer and parallel replicas, both local, both full sorting and hash join
|
||||
@ -65,10 +59,8 @@ nested join with analyzer and parallel replicas, both local, both full sorting a
|
||||
420336 420336 420336 -140112
|
||||
420378 420378 420378 -140126
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
|
||||
<Debug> DefaultCoordinator: Coordination done
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
|
||||
<Debug> WithOrderCoordinator: Coordination done
|
||||
|
||||
nested join with analyzer and parallel replicas, both local, both full sorting and hash join
|
||||
@ -83,8 +75,6 @@ nested join with analyzer and parallel replicas, both local, both full sorting a
|
||||
420336 420336 420336 -140112
|
||||
420378 420378 420378 -140126
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState)
|
||||
<Debug> WithOrderCoordinator: Coordination done
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
|
||||
SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState)
|
||||
<Debug> DefaultCoordinator: Coordination done
|
||||
|
@ -17,6 +17,8 @@ insert into num_1 select number * 2, toString(number * 2) from numbers(1e7);
|
||||
insert into num_2 select number * 3, -number from numbers(1.5e6);
|
||||
"
|
||||
|
||||
PARALLEL_REPLICAS_SETTINGS="allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_local_plan=1"
|
||||
|
||||
##############
|
||||
echo
|
||||
echo "nested join with analyzer and parallel replicas, both global"
|
||||
@ -26,18 +28,14 @@ select * from (select key, value from num_1) l
|
||||
inner join (select key, value from num_2 inner join
|
||||
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r
|
||||
on l.key = r.key order by l.key limit 10 offset 10000
|
||||
SETTINGS enable_analyzer=1,
|
||||
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0"
|
||||
SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0"
|
||||
|
||||
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q "
|
||||
select * from (select key, value from num_1) l
|
||||
inner join (select key, value from num_2 inner join
|
||||
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r
|
||||
on l.key = r.key order by l.key limit 10 offset 10000
|
||||
SETTINGS enable_analyzer=1, send_logs_level='trace',
|
||||
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 |
|
||||
SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, send_logs_level='trace', parallel_replicas_prefer_local_join=0" 2>&1 |
|
||||
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
|
||||
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
|
||||
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
|
||||
@ -51,18 +49,14 @@ select * from (select key, value from num_1) l
|
||||
inner join (select key, value from num_2 inner join
|
||||
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r
|
||||
on l.key = r.key order by l.key limit 10 offset 10000
|
||||
SETTINGS enable_analyzer=1,
|
||||
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0"
|
||||
SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0"
|
||||
|
||||
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q "
|
||||
select * from (select key, value from num_1) l
|
||||
inner join (select key, value from num_2 inner join
|
||||
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r
|
||||
on l.key = r.key order by l.key limit 10 offset 10000
|
||||
SETTINGS enable_analyzer=1, send_logs_level='trace',
|
||||
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 |
|
||||
SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, send_logs_level='trace', parallel_replicas_prefer_local_join=0" 2>&1 |
|
||||
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
|
||||
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
|
||||
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
|
||||
@ -77,18 +71,14 @@ select * from (select key, value from num_1) l
|
||||
inner join (select key, value from num_2 inner join
|
||||
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r
|
||||
on l.key = r.key order by l.key limit 10 offset 10000
|
||||
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0,
|
||||
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'"
|
||||
SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'"
|
||||
|
||||
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q "
|
||||
select * from (select key, value from num_1) l
|
||||
inner join (select key, value from num_2 inner join
|
||||
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r
|
||||
on l.key = r.key order by l.key limit 10 offset 10000
|
||||
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace',
|
||||
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 |
|
||||
SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='full_sorting_merge'" 2>&1 |
|
||||
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
|
||||
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
|
||||
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
|
||||
@ -102,18 +92,14 @@ select * from (select key, value from num_1) l
|
||||
inner join (select key, value from num_2 inner join
|
||||
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r
|
||||
on l.key = r.key order by l.key limit 10 offset 10000
|
||||
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0,
|
||||
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'"
|
||||
SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'"
|
||||
|
||||
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q "
|
||||
select * from (select key, value from num_1) l
|
||||
inner join (select key, value from num_2 inner join
|
||||
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r
|
||||
on l.key = r.key order by l.key limit 10 offset 10000
|
||||
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace',
|
||||
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 |
|
||||
SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='full_sorting_merge'" 2>&1 |
|
||||
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
|
||||
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
|
||||
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
|
||||
@ -127,18 +113,14 @@ select * from (select key, value from num_1) l
|
||||
inner join (select key, value from num_2 inner join
|
||||
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r
|
||||
on l.key = r.key order by l.key limit 10 offset 10000
|
||||
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0,
|
||||
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'"
|
||||
SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='hash'"
|
||||
|
||||
$CLICKHOUSE_CLIENT --max_rows_in_set_to_optimize_join 0 -q "
|
||||
select * from (select key, value from num_1) l
|
||||
inner join (select key, value from num_2 inner join
|
||||
(select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r
|
||||
on l.key = r.key order by l.key limit 10 offset 10000
|
||||
SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace',
|
||||
allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1,
|
||||
cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" 2>&1 |
|
||||
SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='hash'" 2>&1 |
|
||||
grep "executeQuery\|<Debug>.*Coordinator: Coordination done" |
|
||||
grep -o "SELECT.*WithMergeableState)\|<Debug>.*Coordinator: Coordination done" |
|
||||
sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g'
|
||||
|
@ -19,8 +19,20 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x
|
||||
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 allow_experimental_parallel_reading_from_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
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- The same query with cte;
|
||||
with sub1 as (select x, y from tab1 where x != 2),
|
||||
@ -55,8 +67,22 @@ select * from sub5 order by x
|
||||
SETTINGS allow_experimental_parallel_reading_from_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
|
||||
Sorting
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Sorting
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- GROUP BY should work up to WithMergableStage
|
||||
with sub1 as (select x, y from tab1 where x != 2),
|
||||
@ -80,8 +106,22 @@ Expression
|
||||
Sorting
|
||||
Expression
|
||||
MergingAggregated
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Aggregating
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage
|
||||
with sub1 as (select x, y from tab1 where x != 2),
|
||||
@ -118,10 +158,22 @@ Expression
|
||||
Sorting
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- ORDER BY in sub1 : sub1 -> WithMergableStage
|
||||
with sub1 as (select x, y from tab1 where x != 2 order by y),
|
||||
@ -160,12 +212,24 @@ Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- RIGHT JOIN in sub3: sub3 -> WithMergableStage
|
||||
with sub1 as (select x, y from tab1 where x != 2),
|
||||
@ -202,12 +266,24 @@ Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage
|
||||
with sub1 as (select x, y from tab1 where x != 2),
|
||||
@ -242,14 +318,26 @@ Expression
|
||||
Sorting
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- Subqueries for IN allowed
|
||||
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
|
||||
@ -284,8 +372,28 @@ select * from sub5 order by x
|
||||
SETTINGS allow_experimental_parallel_reading_from_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
|
||||
Sorting
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Sorting
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
CreatingSets
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
CreatingSet
|
||||
Expression
|
||||
Filter
|
||||
ReadFromSystemNumbers
|
||||
Expression
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
ReadFromMemoryStorage
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- Subqueries for IN are not allowed
|
||||
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
|
||||
@ -333,10 +441,18 @@ Expression
|
||||
Expression
|
||||
Filter
|
||||
ReadFromSystemNumbers
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
ReadFromMergeTree
|
||||
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 allow_experimental_parallel_reading_from_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;
|
||||
@ -356,8 +472,22 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x
|
||||
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 allow_experimental_parallel_reading_from_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
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- The same query with cte;
|
||||
with sub1 as (select x, y from tab1 where x != 2),
|
||||
@ -392,8 +522,24 @@ select * from sub5 order by x
|
||||
SETTINGS allow_experimental_parallel_reading_from_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
|
||||
Sorting
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Sorting
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- GROUP BY should work up to WithMergableStage
|
||||
with sub1 as (select x, y from tab1 where x != 2),
|
||||
@ -417,8 +563,24 @@ Expression
|
||||
Sorting
|
||||
Expression
|
||||
MergingAggregated
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Aggregating
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage
|
||||
with sub1 as (select x, y from tab1 where x != 2),
|
||||
@ -455,10 +617,23 @@ Expression
|
||||
Sorting
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- ORDER BY in sub1 : sub1 -> WithMergableStage
|
||||
with sub1 as (select x, y from tab1 where x != 2 order by y),
|
||||
@ -497,12 +672,24 @@ Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- RIGHT JOIN in sub3: sub3 -> WithMergableStage
|
||||
with sub1 as (select x, y from tab1 where x != 2),
|
||||
@ -539,12 +726,24 @@ Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- RIGHT JOIN in sub5: sub5 -> WithMergableStage
|
||||
with sub1 as (select x, y from tab1 where x != 2),
|
||||
@ -579,14 +778,26 @@ Expression
|
||||
Sorting
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- Subqueries for IN allowed
|
||||
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
|
||||
@ -621,8 +832,30 @@ select * from sub5 order by x
|
||||
SETTINGS allow_experimental_parallel_reading_from_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
|
||||
Sorting
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Sorting
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
Join
|
||||
Expression
|
||||
CreatingSets
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
CreatingSet
|
||||
Expression
|
||||
Filter
|
||||
ReadFromSystemNumbers
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
--
|
||||
-- Subqueries for IN are not allowed
|
||||
with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)),
|
||||
@ -670,7 +903,15 @@ Expression
|
||||
Expression
|
||||
Filter
|
||||
ReadFromSystemNumbers
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Union
|
||||
Expression
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
ReadFromMergeTree
|
||||
Expression
|
||||
ReadFromRemoteParallelReplicas
|
||||
|
@ -10,6 +10,8 @@ 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;
|
||||
|
||||
{% for use_global_in in [0, 1] -%}
|
||||
|
||||
-- { echoOn }
|
||||
|
@ -0,0 +1 @@
|
||||
default url_na_log 1 130000 130
|
@ -21,11 +21,21 @@ SELECT
|
||||
FROM numbers(130000)
|
||||
SETTINGS max_insert_block_size = 200000;
|
||||
|
||||
SET max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000;
|
||||
|
||||
EXPLAIN ESTIMATE
|
||||
SELECT count()
|
||||
FROM url_na_log
|
||||
PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20')
|
||||
SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000;
|
||||
SETTINGS parallel_replicas_local_plan=0;
|
||||
|
||||
-- here parallel replicas uses local snapshot as working set
|
||||
-- so, the estimation can be done
|
||||
EXPLAIN ESTIMATE
|
||||
SELECT count()
|
||||
FROM url_na_log
|
||||
PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20')
|
||||
SETTINGS allow_experimental_analyzer=1, parallel_replicas_local_plan=1;
|
||||
|
||||
DROP POLICY url_na_log_policy0 ON url_na_log;
|
||||
DROP TABLE url_na_log;
|
||||
|
@ -0,0 +1,21 @@
|
||||
DROP TABLE IF EXISTS test_00808;
|
||||
|
||||
CREATE TABLE test_00808
|
||||
(
|
||||
`date` Date,
|
||||
`id` Int8,
|
||||
`name` String,
|
||||
`value` Int64,
|
||||
`sign` Int8
|
||||
)
|
||||
ENGINE = CollapsingMergeTree(sign)
|
||||
ORDER BY (id, date);
|
||||
|
||||
INSERT INTO test_00808 VALUES('2000-01-01', 1, 'test string 1', 1, 1);
|
||||
INSERT INTO test_00808 VALUES('2000-01-01', 2, 'test string 2', 2, 1);
|
||||
|
||||
SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost';
|
||||
|
||||
SELECT * FROM (SELECT * FROM test_00808 FINAL) WHERE id = 1; -- { serverError SUPPORT_IS_DISABLED }
|
||||
|
||||
DROP TABLE test_00808;
|
@ -0,0 +1,100 @@
|
||||
0 18 9899
|
||||
0 18 9898
|
||||
0 18 9897
|
||||
0 18 9896
|
||||
0 18 9895
|
||||
0 18 9894
|
||||
0 18 9893
|
||||
0 18 9892
|
||||
0 18 9891
|
||||
0 18 9890
|
||||
0 18 9889
|
||||
0 18 9888
|
||||
0 18 9887
|
||||
0 18 9886
|
||||
0 18 9885
|
||||
0 18 9884
|
||||
0 18 9883
|
||||
0 18 9882
|
||||
0 18 9881
|
||||
0 18 9880
|
||||
0 18 9879
|
||||
0 18 9878
|
||||
0 18 9877
|
||||
0 18 9876
|
||||
0 18 9875
|
||||
0 18 9874
|
||||
0 18 9873
|
||||
0 18 9872
|
||||
0 18 9871
|
||||
0 18 9870
|
||||
0 18 9869
|
||||
0 18 9868
|
||||
0 18 9867
|
||||
0 18 9866
|
||||
0 18 9865
|
||||
0 18 9864
|
||||
0 18 9863
|
||||
0 18 9862
|
||||
0 18 9861
|
||||
0 18 9860
|
||||
0 18 9859
|
||||
0 18 9858
|
||||
0 18 9857
|
||||
0 18 9856
|
||||
0 18 9855
|
||||
0 18 9854
|
||||
0 18 9853
|
||||
0 18 9852
|
||||
0 18 9851
|
||||
0 18 9850
|
||||
0 18 9849
|
||||
0 18 9848
|
||||
0 18 9847
|
||||
0 18 9846
|
||||
0 18 9845
|
||||
0 18 9844
|
||||
0 18 9843
|
||||
0 18 9842
|
||||
0 18 9841
|
||||
0 18 9840
|
||||
0 18 9839
|
||||
0 18 9838
|
||||
0 18 9837
|
||||
0 18 9836
|
||||
0 18 9835
|
||||
0 18 9834
|
||||
0 18 9833
|
||||
0 18 9832
|
||||
0 18 9831
|
||||
0 18 9830
|
||||
0 18 9829
|
||||
0 18 9828
|
||||
0 18 9827
|
||||
0 18 9826
|
||||
0 18 9825
|
||||
0 18 9824
|
||||
0 18 9823
|
||||
0 18 9822
|
||||
0 18 9821
|
||||
0 18 9820
|
||||
0 18 9819
|
||||
0 18 9818
|
||||
0 18 9817
|
||||
0 18 9816
|
||||
0 18 9815
|
||||
0 18 9814
|
||||
0 18 9813
|
||||
0 18 9812
|
||||
0 18 9811
|
||||
0 18 9810
|
||||
0 18 9809
|
||||
0 18 9808
|
||||
0 18 9807
|
||||
0 18 9806
|
||||
0 18 9805
|
||||
0 18 9804
|
||||
0 18 9803
|
||||
0 18 9802
|
||||
0 18 9801
|
||||
0 18 9800
|
@ -0,0 +1,14 @@
|
||||
DROP TABLE IF EXISTS test;
|
||||
|
||||
CREATE TABLE test (k UInt64, v String)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY k
|
||||
SETTINGS index_granularity=1;
|
||||
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(10_000);
|
||||
|
||||
SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost';
|
||||
|
||||
SELECT 0, materialize(18), k FROM test PREWHERE toNullable(toNullable(11)) WHERE toNullable(11) ORDER BY k DESC NULLS LAST LIMIT 100, 100 SETTINGS optimize_read_in_order = 1, merge_tree_min_rows_for_concurrent_read = 9223372036854775806, max_threads = 1;
|
||||
|
||||
-- DROP TABLE test;
|
@ -0,0 +1,2 @@
|
||||
9
|
||||
9
|
@ -0,0 +1,14 @@
|
||||
drop table if exists test_left;
|
||||
drop table if exists test_right;
|
||||
|
||||
CREATE TABLE test_left (a Int64, b String, c LowCardinality(String)) ENGINE = MergeTree() ORDER BY a;
|
||||
CREATE TABLE test_right (a Int64, b String, c LowCardinality(String)) ENGINE = MergeTree() ORDER BY a;
|
||||
|
||||
INSERT INTO test_left SELECT number % 10000, number % 10000, number % 10000 FROM numbers(100000);
|
||||
INSERT INTO test_right SELECT number % 10 , number % 10, number % 10 FROM numbers(10000);
|
||||
|
||||
SELECT MAX(test_right.a) FROM test_left INNER JOIN test_right on test_left.b = test_right.b SETTINGS allow_experimental_join_right_table_sorting=true;
|
||||
SELECT MAX(test_right.a) FROM test_left LEFT JOIN test_right on test_left.b = test_right.b SETTINGS allow_experimental_join_right_table_sorting=true;
|
||||
|
||||
drop table test_left;
|
||||
drop table test_right;
|
@ -0,0 +1,20 @@
|
||||
300
|
||||
299
|
||||
298
|
||||
297
|
||||
296
|
||||
295
|
||||
294
|
||||
293
|
||||
292
|
||||
291
|
||||
290
|
||||
289
|
||||
288
|
||||
287
|
||||
286
|
||||
285
|
||||
284
|
||||
283
|
||||
282
|
||||
281
|
@ -0,0 +1,18 @@
|
||||
DROP TABLE IF EXISTS view1;
|
||||
DROP TABLE IF EXISTS table1;
|
||||
CREATE TABLE table1 (number UInt64) ENGINE=MergeTree ORDER BY number SETTINGS index_granularity=1;
|
||||
INSERT INTO table1 SELECT number FROM numbers(1, 300);
|
||||
CREATE VIEW view1 AS SELECT number FROM table1;
|
||||
|
||||
SELECT *
|
||||
FROM
|
||||
(
|
||||
SELECT *
|
||||
FROM view1
|
||||
)
|
||||
ORDER BY number DESC
|
||||
LIMIT 20
|
||||
SETTINGS cluster_for_parallel_replicas = 'parallel_replicas', allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree = 1, parallel_replicas_local_plan = 1;
|
||||
|
||||
DROP TABLE view1;
|
||||
DROP TABLE table1;
|
27
tests/queries/0_stateless/03236_squashing_high_memory.sql
Normal file
27
tests/queries/0_stateless/03236_squashing_high_memory.sql
Normal file
@ -0,0 +1,27 @@
|
||||
-- Tags: no-fasttest, no-asan, no-tsan, no-msan, no-ubsan
|
||||
-- reason: test requires too many rows to read
|
||||
|
||||
SET max_rows_to_read = '501G';
|
||||
|
||||
DROP TABLE IF EXISTS id_values;
|
||||
|
||||
DROP TABLE IF EXISTS test_table;
|
||||
|
||||
CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS
|
||||
SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2;
|
||||
|
||||
SET max_memory_usage = '1G';
|
||||
|
||||
CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS
|
||||
SELECT id_values.id1 AS id,
|
||||
string_values.string_val1 AS string_val1,
|
||||
string_values.string_val2 AS string_val2
|
||||
FROM id_values
|
||||
JOIN (SELECT arrayJoin(range(10)) AS id1,
|
||||
'qwe' AS string_val1,
|
||||
'asd' AS string_val2) AS string_values
|
||||
ON id_values.id1 = string_values.id1
|
||||
SETTINGS join_algorithm = 'hash';
|
||||
|
||||
DROP TABLE IF EXISTS id_values;
|
||||
DROP TABLE IF EXISTS test_table;
|
@ -10,6 +10,8 @@ http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-18 http:/
|
||||
http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-19 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny
|
||||
http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-20 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny
|
||||
1
|
||||
SortingAggregatedTransform
|
||||
MergingAggregatedBucketTransform
|
||||
GroupingAggregatedTransform
|
||||
FinishAggregatingInOrderTransform
|
||||
FinalizeAggregatedTransform
|
||||
AggregatingInOrderTransform
|
||||
MergeTreeSelect
|
||||
|
@ -55,7 +55,7 @@ test2() {
|
||||
test3() {
|
||||
$CLICKHOUSE_CLIENT -q "
|
||||
SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost';
|
||||
SET max_threads = 16, prefer_localhost_replica = 1, read_in_order_two_level_merge_threshold = 1000, query_plan_aggregation_in_order = 1, distributed_aggregation_memory_efficient = 1;
|
||||
SET max_threads = 16, read_in_order_two_level_merge_threshold = 1000, query_plan_aggregation_in_order = 1, distributed_aggregation_memory_efficient = 1;
|
||||
|
||||
SELECT replaceRegexpOne(explain, '^ *(\w+).*', '\\1')
|
||||
FROM (
|
||||
@ -64,7 +64,7 @@ test3() {
|
||||
FROM test.hits
|
||||
WHERE CounterID = 1704509 AND UserID = 4322253409885123546
|
||||
GROUP BY URL, EventDate
|
||||
SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3
|
||||
SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, parallel_replicas_local_plan=1
|
||||
)
|
||||
WHERE explain LIKE '%Aggr%Transform%' OR explain LIKE '%InOrder%'"
|
||||
}
|
||||
|
@ -31,6 +31,7 @@ allow_experimental_statistics
|
||||
allow_experimental_time_series_table
|
||||
allow_experimental_undrop_table_query
|
||||
allow_experimental_usearch_index
|
||||
allow_experimental_join_right_table_sorting
|
||||
allow_get_client_http_header
|
||||
allow_introspection_functions
|
||||
allow_materialized_view_with_bad_select
|
||||
|
Loading…
Reference in New Issue
Block a user