Merge remote-tracking branch 'origin/master' into fix-pr-max-execution-timeout-leaf

This commit is contained in:
Igor Nikonov 2024-09-12 08:17:44 +00:00
commit 60d505b59a
51 changed files with 1204 additions and 309 deletions

View File

@ -948,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) \

View File

@ -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",

View File

@ -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",
@ -494,12 +497,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,
@ -525,21 +528,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(
@ -549,7 +678,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);
@ -559,7 +689,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(
@ -655,6 +786,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;
}
}
}

View File

@ -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,

View File

@ -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);
}
}
}

View File

@ -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 {};
}

View 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));
}
}

View File

@ -0,0 +1,9 @@
#pragma once
namespace DB
{
class QueryPlan;
class Block;
void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects);
}

View File

@ -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,

View File

@ -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));

View 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};
}
}

View 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);
}

View File

@ -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.

View File

@ -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;
};
}

View File

@ -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));

View File

@ -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;
};
}

View File

@ -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));

View File

@ -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)

View File

@ -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,
),

View File

@ -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:

View File

@ -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

View File

@ -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>

View File

@ -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,
},
)

View File

@ -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

View File

@ -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;

View File

@ -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;

View File

@ -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) ===============';

View File

@ -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;

View File

@ -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`

View File

@ -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;

View File

@ -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" }'
}

View File

@ -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

View File

@ -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"
}

View File

@ -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"'

View File

@ -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

View File

@ -1,4 +1,4 @@
100 4950
10000 49995000
1
89
90
@ -6,3 +6,9 @@
92
93
1
93
92
91
90
89
1

View File

@ -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;

View File

@ -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

View File

@ -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'

View File

@ -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

View File

@ -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 }

View File

@ -0,0 +1 @@
default url_na_log 1 130000 130

View File

@ -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;

View File

@ -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;

View File

@ -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

View File

@ -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;

View File

@ -0,0 +1,20 @@
300
299
298
297
296
295
294
293
292
291
290
289
288
287
286
285
284
283
282
281

View File

@ -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;

View File

@ -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

View File

@ -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%'"
}